You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/04/18 02:41:02 UTC

[GitHub] [lucene] mocobeta commented on a diff in pull request #805: LUCENE-10493: factor out Viterbi algorithm and share it between kuromoji and nori

mocobeta commented on code in PR #805:
URL: https://github.com/apache/lucene/pull/805#discussion_r851846785


##########
lucene/analysis/kuromoji/src/java/org/apache/lucene/analysis/ja/ViterbiNBest.java:
##########
@@ -0,0 +1,1253 @@
+/*
+ * 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.ja;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import org.apache.lucene.analysis.ja.dict.CharacterDefinition;
+import org.apache.lucene.analysis.ja.dict.JaMorphData;
+import org.apache.lucene.analysis.ja.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.ja.dict.UnknownDictionary;
+import org.apache.lucene.analysis.ja.dict.UserDictionary;
+import org.apache.lucene.analysis.morph.ConnectionCosts;
+import org.apache.lucene.analysis.morph.Dictionary;
+import org.apache.lucene.analysis.morph.GraphvizFormatter;
+import org.apache.lucene.analysis.morph.TokenInfoFST;
+import org.apache.lucene.analysis.morph.TokenType;
+import org.apache.lucene.analysis.morph.Viterbi;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.fst.FST;
+
+/**
+ * {@link org.apache.lucene.analysis.morph.Viterbi} subclass for Japanese morphological analysis.
+ * This also performs n-best path calculation
+ */
+final class ViterbiNBest
+    extends org.apache.lucene.analysis.morph.Viterbi<Token, ViterbiNBest.PositionNBest> {
+
+  private final EnumMap<TokenType, Dictionary<? extends JaMorphData>> dictionaryMap =
+      new EnumMap<>(TokenType.class);
+
+  private final UnknownDictionary unkDictionary;
+  private final CharacterDefinition characterDefinition;
+  private final UserDictionary userDictionary;
+
+  private final boolean discardPunctuation;
+  private final boolean searchMode;
+  private final boolean extendedMode;
+  private final boolean outputCompounds;
+
+  // Allowable cost difference for N-best output:
+  private int nBestCost = 0;
+
+  Lattice lattice = null;
+
+  private GraphvizFormatter<JaMorphData> dotOut;
+
+  ViterbiNBest(
+      TokenInfoFST fst,
+      FST.BytesReader fstReader,
+      TokenInfoDictionary dictionary,
+      TokenInfoFST userFST,
+      FST.BytesReader userFSTReader,
+      UserDictionary userDictionary,
+      ConnectionCosts costs,
+      Class<PositionNBest> positionImpl,
+      UnknownDictionary unkDictionary,
+      CharacterDefinition characterDefinition,
+      boolean discardPunctuation,
+      boolean searchMode,
+      boolean extendedMode,
+      boolean outputCompounds) {
+    super(fst, fstReader, dictionary, userFST, userFSTReader, userDictionary, costs, positionImpl);
+    this.unkDictionary = unkDictionary;
+    this.characterDefinition = characterDefinition;
+    this.userDictionary = userDictionary;
+    this.discardPunctuation = discardPunctuation;
+    this.searchMode = searchMode;
+    this.extendedMode = extendedMode;
+    this.outputCompounds = outputCompounds;
+    dictionaryMap.put(TokenType.KNOWN, dictionary);
+    dictionaryMap.put(TokenType.UNKNOWN, unkDictionary);
+    dictionaryMap.put(TokenType.USER, userDictionary);
+  }
+
+  @Override
+  protected boolean shouldSkipProcessUnknownWord(int unknownWordEndIndex, Position posData) {
+    return !searchMode && super.shouldSkipProcessUnknownWord(unknownWordEndIndex, posData);
+  }
+
+  private static final int SEARCH_MODE_KANJI_LENGTH = 2;
+  private static final int SEARCH_MODE_OTHER_LENGTH = 7; // Must be >= SEARCH_MODE_KANJI_LENGTH
+  private static final int SEARCH_MODE_KANJI_PENALTY = 3000;
+  private static final int SEARCH_MODE_OTHER_PENALTY = 1700;
+
+  @Override
+  protected int computePenalty(int pos, int length) throws IOException {
+    if (length > SEARCH_MODE_KANJI_LENGTH) {
+      boolean allKanji = true;
+      // check if node consists of only kanji
+      final int endPos = pos + length;
+      for (int pos2 = pos; pos2 < endPos; pos2++) {
+        if (!characterDefinition.isKanji((char) buffer.get(pos2))) {
+          allKanji = false;
+          break;
+        }
+      }
+      if (allKanji) { // Process only Kanji keywords
+        return (length - SEARCH_MODE_KANJI_LENGTH) * SEARCH_MODE_KANJI_PENALTY;
+      } else if (length > SEARCH_MODE_OTHER_LENGTH) {
+        return (length - SEARCH_MODE_OTHER_LENGTH) * SEARCH_MODE_OTHER_PENALTY;
+      }
+    }
+    return 0;
+  }
+
+  // Returns the added cost that a 2nd best segmentation is
+  // allowed to have.  Ie, if we see path with cost X,
+  // ending in a compound word, and this method returns
+  // threshold > 0, then we will also find the 2nd best
+  // segmentation and if its path score is within this
+  // threshold of X, we'll include it in the output:
+  private int computeSecondBestThreshold(int pos, int length) throws IOException {
+    // TODO: maybe we do something else here, instead of just
+    // using the penalty...?  EG we can be more aggressive on
+    // when to also test for 2nd best path
+    return computePenalty(pos, length);
+  }
+
+  @Override
+  protected int processUnknownWord(boolean anyMatches, Position posData) throws IOException {
+    final char firstCharacter = (char) buffer.get(pos);
+    if (!anyMatches || characterDefinition.isInvoke(firstCharacter)) {
+
+      // Find unknown match:
+      final int characterId = characterDefinition.getCharacterClass(firstCharacter);
+      final boolean isPunct = isPunctuation(firstCharacter);
+
+      // NOTE: copied from UnknownDictionary.lookup:
+      int unknownWordLength;
+      if (!characterDefinition.isGroup(firstCharacter)) {
+        unknownWordLength = 1;
+      } else {
+        // Extract unknown word. Characters with the same character class are considered to be
+        // part of unknown word
+        unknownWordLength = 1;
+        for (int posAhead = pos + 1; unknownWordLength < MAX_UNKNOWN_WORD_LENGTH; posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          if (characterId == characterDefinition.getCharacterClass((char) ch)
+              && isPunctuation((char) ch) == isPunct) {
+            unknownWordLength++;
+          } else {
+            break;
+          }
+        }
+      }
+
+      unkDictionary.lookupWordIds(
+          characterId, wordIdRef); // characters in input text are supposed to be the same
+      if (VERBOSE) {
+        System.out.println(
+            "    UNKNOWN word len=" + unknownWordLength + " " + wordIdRef.length + " wordIDs");
+      }
+      for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+        add(
+            unkDictionary.getMorphAttributes(),
+            posData,
+            pos,
+            posData.getPos() + unknownWordLength,
+            wordIdRef.ints[wordIdRef.offset + ofs],
+            TokenType.UNKNOWN,
+            false);
+      }
+
+      return unknownWordLength;
+    }
+    return 0;
+  }
+
+  void setGraphvizFormatter(GraphvizFormatter<JaMorphData> dotOut) {
+    this.dotOut = dotOut;
+  }
+
+  @Override
+  protected void backtrace(Position endPosData, int fromIDX) throws IOException {
+    final int endPos = endPosData.getPos();
+
+    /**
+     * LUCENE-10059: If the endPos is the same as lastBackTracePos, we don't want to backtrace to
+     * avoid an assertion error {@link RollingCharBuffer#get(int)} when it tries to generate an
+     * empty buffer
+     */
+    if (endPos == lastBackTracePos) {

Review Comment:
   +1 I think this should be ported to Nori.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org