You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2021/03/19 01:57:11 UTC

[lucene] branch main updated: LUCENE-9852: Make Hunspell thread-safe (#24)

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

rmuir 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 28edbf8  LUCENE-9852: Make Hunspell thread-safe (#24)
28edbf8 is described below

commit 28edbf8fc6804b883a88dfda4ca943120d6c77b3
Author: Peter Gromov <pe...@jetbrains.com>
AuthorDate: Fri Mar 19 02:57:03 2021 +0100

    LUCENE-9852: Make Hunspell thread-safe (#24)
---
 .../apache/lucene/analysis/hunspell/Hunspell.java  |  3 +-
 .../apache/lucene/analysis/hunspell/Stemmer.java   | 62 +++++++---------------
 .../lucene/analysis/hunspell/TestPerformance.java  | 61 ++++++++++++++++-----
 3 files changed, 66 insertions(+), 60 deletions(-)

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 4eadcd1..0c9849e 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
@@ -48,8 +48,7 @@ import org.apache.lucene.util.IntsRef;
  *   <li>PHONE affix file option for suggestions
  * </ul>
  *
- * <p>The objects of this class are not thread-safe (but a single underlying Dictionary can be
- * shared by multiple spell-checkers in different threads).
+ * <p>The objects of this class are thread-safe.
  */
 public class Hunspell {
   static final long SUGGEST_TIME_LIMIT = 250;
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 488adfd..012f8bb 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
@@ -33,11 +33,6 @@ import org.apache.lucene.util.fst.FST;
  */
 final class Stemmer {
   private final Dictionary dictionary;
-  private final StringBuilder segment = new StringBuilder();
-
-  // used for normalization
-  private final StringBuilder scratchSegment = new StringBuilder();
-  private char[] scratchBuffer = new char[32];
 
   // it's '1' if we have no stem exceptions, otherwise every other form
   // is really an ID pointing to the exception table
@@ -50,16 +45,6 @@ final class Stemmer {
    */
   public Stemmer(Dictionary dictionary) {
     this.dictionary = dictionary;
-    prefixReader = dictionary.prefixes == null ? null : dictionary.prefixes.getBytesReader();
-    suffixReader = dictionary.suffixes == null ? null : dictionary.suffixes.getBytesReader();
-    for (int level = 0; level < 3; level++) {
-      if (dictionary.prefixes != null) {
-        prefixArcs[level] = new FST.Arc<>();
-      }
-      if (dictionary.suffixes != null) {
-        suffixArcs[level] = new FST.Arc<>();
-      }
-    }
     formStep = dictionary.formStep();
   }
 
@@ -82,11 +67,11 @@ final class Stemmer {
   public List<CharsRef> stem(char[] word, int length) {
 
     if (dictionary.mayNeedInputCleaning()) {
-      scratchSegment.setLength(0);
-      scratchSegment.append(word, 0, length);
+      CharsRef scratchSegment = new CharsRef(word, 0, length);
       if (dictionary.needsInputCleaning(scratchSegment)) {
-        CharSequence cleaned = dictionary.cleanInput(scratchSegment, segment);
-        scratchBuffer = ArrayUtil.grow(scratchBuffer, cleaned.length());
+        StringBuilder segment = new StringBuilder();
+        dictionary.cleanInput(scratchSegment, segment);
+        char[] scratchBuffer = new char[segment.length()];
         length = segment.length();
         segment.getChars(0, length, scratchBuffer, 0);
         word = scratchBuffer;
@@ -122,8 +107,8 @@ final class Stemmer {
   }
 
   boolean varyCase(char[] word, int length, WordCase wordCase, CaseVariationProcessor processor) {
+    char[] titleBuffer = wordCase == WordCase.UPPER ? caseFoldTitle(word, length) : null;
     if (wordCase == WordCase.UPPER) {
-      caseFoldTitle(word, length);
       char[] aposCase = capitalizeAfterApostrophe(titleBuffer, length);
       if (aposCase != null && !processor.process(aposCase, length, wordCase)) {
         return false;
@@ -140,7 +125,7 @@ final class Stemmer {
       return true;
     }
 
-    caseFoldLower(wordCase == WordCase.UPPER ? titleBuffer : word, length);
+    char[] lowerBuffer = caseFoldLower(titleBuffer != null ? titleBuffer : word, length);
     if (!processor.process(lowerBuffer, length, wordCase)) {
       return false;
     }
@@ -152,10 +137,6 @@ final class Stemmer {
     return true;
   }
 
-  // temporary buffers for case variants
-  private char[] lowerBuffer = new char[8];
-  private char[] titleBuffer = new char[8];
-
   /** returns EXACT_CASE,TITLE_CASE, or UPPER_CASE type for the word */
   WordCase caseOf(char[] word, int length) {
     if (dictionary.ignoreCase || length == 0 || Character.isLowerCase(word[0])) {
@@ -166,19 +147,21 @@ final class Stemmer {
   }
 
   /** folds titlecase variant of word to titleBuffer */
-  private void caseFoldTitle(char[] word, int length) {
-    titleBuffer = ArrayUtil.grow(titleBuffer, length);
+  private char[] caseFoldTitle(char[] word, int length) {
+    char[] titleBuffer = new char[length];
     System.arraycopy(word, 0, titleBuffer, 0, length);
     for (int i = 1; i < length; i++) {
       titleBuffer[i] = dictionary.caseFold(titleBuffer[i]);
     }
+    return titleBuffer;
   }
 
   /** folds lowercase variant of word (title cased) to lowerBuffer */
-  private void caseFoldLower(char[] word, int length) {
-    lowerBuffer = ArrayUtil.grow(lowerBuffer, length);
+  private char[] caseFoldLower(char[] word, int length) {
+    char[] lowerBuffer = new char[length];
     System.arraycopy(word, 0, lowerBuffer, 0, length);
     lowerBuffer[0] = dictionary.caseFold(lowerBuffer[0]);
+    return lowerBuffer;
   }
 
   // Special prefix handling for Catalan, French, Italian:
@@ -315,7 +298,7 @@ final class Stemmer {
     String exception = stemException(morphDataId);
 
     if (dictionary.oconv != null) {
-      scratchSegment.setLength(0);
+      StringBuilder scratchSegment = new StringBuilder();
       if (exception != null) {
         scratchSegment.append(exception);
       } else {
@@ -334,16 +317,6 @@ final class Stemmer {
     }
   }
 
-  // some state for traversing FSTs
-  private final FST.BytesReader prefixReader;
-  private final FST.BytesReader suffixReader;
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private final FST.Arc<IntsRef>[] prefixArcs = new FST.Arc[3];
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private final FST.Arc<IntsRef>[] suffixArcs = new FST.Arc[3];
-
   /**
    * Generates a list of stems for the provided word
    *
@@ -372,15 +345,16 @@ final class Stemmer {
       boolean doPrefix,
       boolean previousWasPrefix,
       RootProcessor processor) {
+    FST.Arc<IntsRef> arc = new FST.Arc<>();
     if (doPrefix && dictionary.prefixes != null) {
       FST<IntsRef> fst = dictionary.prefixes;
-      FST.Arc<IntsRef> arc = prefixArcs[recursionDepth];
+      FST.BytesReader reader = fst.getBytesReader();
       fst.getFirstArc(arc);
       IntsRef output = fst.outputs.getNoOutput();
       int limit = dictionary.fullStrip ? length + 1 : length;
       for (int i = 0; i < limit; i++) {
         if (i > 0) {
-          output = Dictionary.nextArc(fst, arc, prefixReader, output, word[offset + i - 1]);
+          output = Dictionary.nextArc(fst, arc, reader, output, word[offset + i - 1]);
           if (output == null) {
             break;
           }
@@ -423,13 +397,13 @@ final class Stemmer {
 
     if (dictionary.suffixes != null) {
       FST<IntsRef> fst = dictionary.suffixes;
-      FST.Arc<IntsRef> arc = suffixArcs[recursionDepth];
+      FST.BytesReader reader = fst.getBytesReader();
       fst.getFirstArc(arc);
       IntsRef output = fst.outputs.getNoOutput();
       int limit = dictionary.fullStrip ? 0 : 1;
       for (int i = length; i >= limit; i--) {
         if (i < length) {
-          output = Dictionary.nextArc(fst, arc, suffixReader, output, word[offset + i]);
+          output = Dictionary.nextArc(fst, arc, reader, output, word[offset + i]);
           if (output == null) {
             break;
           }
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 ffe3ae9..4e2d300 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
@@ -28,11 +28,15 @@ import java.nio.file.Paths;
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
 import org.junit.Assume;
 import org.junit.AssumptionViolatedException;
 import org.junit.BeforeClass;
@@ -106,26 +110,55 @@ public class TestPerformance extends LuceneTestCase {
     Dictionary dictionary = loadDictionary(code);
 
     List<String> words = loadWords(code, wordCount, dictionary);
+    List<String> halfWords = words.subList(0, words.size() / 2);
 
     Stemmer stemmer = new Stemmer(dictionary);
     Hunspell speller = new Hunspell(dictionary, TimeoutPolicy.NO_TIMEOUT, () -> {});
-    measure(
-        "Stemming " + code,
-        blackHole -> {
-          for (String word : words) {
-            blackHole.accept(stemmer.stem(word));
-          }
-        });
-    measure(
-        "Spellchecking " + code,
-        blackHole -> {
-          for (String word : words) {
-            blackHole.accept(speller.spell(word));
-          }
-        });
+    int cpus = Runtime.getRuntime().availableProcessors();
+    ExecutorService executor =
+        Executors.newFixedThreadPool(cpus, new NamedThreadFactory("hunspellStemming-"));
+
+    try {
+      measure("Stemming " + code, blackHole -> stemWords(words, stemmer, blackHole));
+
+      measure(
+          "Multi-threaded stemming " + code,
+          blackHole -> {
+            List<Future<?>> futures = new ArrayList<>();
+            for (int i = 0; i < cpus; i++) {
+              Stemmer localStemmer = new Stemmer(dictionary);
+              futures.add(executor.submit(() -> stemWords(halfWords, localStemmer, blackHole)));
+            }
+            try {
+              for (Future<?> future : futures) {
+                future.get();
+              }
+            } catch (Exception e) {
+              throw new RuntimeException(e);
+            }
+          });
+
+      measure(
+          "Spellchecking " + code,
+          blackHole -> {
+            for (String word : words) {
+              blackHole.accept(speller.spell(word));
+            }
+          });
+    } finally {
+      executor.shutdown();
+      assertTrue(executor.awaitTermination(1, TimeUnit.MINUTES));
+    }
+
     System.out.println();
   }
 
+  private void stemWords(List<String> words, Stemmer stemmer, Consumer<Object> blackHole) {
+    for (String word : words) {
+      blackHole.accept(stemmer.stem(word));
+    }
+  }
+
   private void checkSuggestionPerformance(String code, int wordCount) throws Exception {
     Dictionary dictionary = loadDictionary(code);
     Hunspell speller = new Hunspell(dictionary, TimeoutPolicy.THROW_EXCEPTION, () -> {});