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 2012/01/08 09:49:13 UTC

svn commit: r1228810 - in /lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src: java/org/apache/lucene/analysis/kuromoji/ java/org/apache/lucene/analysis/kuromoji/dict/ java/org/apache/lucene/analysis/kuromoji/trie/ java/org/apache/lucene/anal...

Author: rmuir
Date: Sun Jan  8 08:49:12 2012
New Revision: 1228810

URL: http://svn.apache.org/viewvc?rev=1228810&view=rev
Log:
LUCENE-3305: switch to FST

Added:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$fst.dat   (with props)
Removed:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/trie/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$trie.dat
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/
Modified:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/BinaryDictionary.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$targetMap.dat
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary$targetMap.dat
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/BinaryDictionaryWriter.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/Segmenter.java Sun Jan  8 08:49:12 2012
@@ -157,7 +157,12 @@ public class Segmenter {
   public List<Token> doTokenize(int offset, char[] sentence, int sentenceOffset, int sentenceLength) {
     ArrayList<Token> result = new ArrayList<Token>();
     
-    ViterbiNode[][][] lattice = viterbi.build(sentence, sentenceOffset, sentenceLength);
+    ViterbiNode[][][] lattice;
+    try {
+      lattice = viterbi.build(sentence, sentenceOffset, sentenceLength);
+    } catch (IOException impossible) {
+      throw new RuntimeException(impossible);
+    }
     List<ViterbiNode> bestPath = viterbi.search(lattice);
     for (ViterbiNode node : bestPath) {
       int wordId = node.getWordId();
@@ -173,7 +178,12 @@ public class Segmenter {
   
   /** returns a Graphviz String */
   public String debugTokenize(String text) {
-    ViterbiNode[][][] lattice = this.viterbi.build(text.toCharArray(), 0, text.length());
+    ViterbiNode[][][] lattice;
+    try {
+      lattice = this.viterbi.build(text.toCharArray(), 0, text.length());
+    } catch (IOException impossible) {
+      throw new RuntimeException(impossible);
+    }
     List<ViterbiNode> bestPath = this.viterbi.search(lattice);
     
     return new GraphvizFormatter(ConnectionCosts.getInstance())

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/BinaryDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/BinaryDictionary.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/BinaryDictionary.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/BinaryDictionary.java Sun Jan  8 08:49:12 2012
@@ -67,8 +67,10 @@ public abstract class BinaryDictionary i
           j += in.readVInt();
         } else {
           final int[] a = new int[len];
+          int accum = 0;
           for (int i = 0; i < len; i++) {
-            a[i] = in.readVInt();
+            accum += in.readVInt();
+            a[i] = accum;
           }
           targetMap[j] = a;
           j++;

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java Sun Jan  8 08:49:12 2012
@@ -21,37 +21,39 @@ import java.io.InputStream;
 import java.io.IOException;
 import java.io.FileNotFoundException;
 
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
-
+import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
 
 public final class TokenInfoDictionary extends BinaryDictionary {
 
-  public static final String TRIE_FILENAME_SUFFIX = "$trie.dat";
+  public static final String FST_FILENAME_SUFFIX = "$fst.dat";
 
-  private final DoubleArrayTrie trie;
+  private final TokenInfoFST fst;
   
   private TokenInfoDictionary() throws IOException {
     super();
     InputStream is = null;
-    DoubleArrayTrie trie = null;
+    FST<Long> fst = null;
     try {
-      is = getClass().getResourceAsStream(getClass().getSimpleName() + TRIE_FILENAME_SUFFIX);
+      is = getClass().getResourceAsStream(getClass().getSimpleName() + FST_FILENAME_SUFFIX);
       if (is == null)
-        throw new FileNotFoundException("Not in classpath: " + getClass().getName().replace('.','/') + TRIE_FILENAME_SUFFIX);
-      trie = new DoubleArrayTrie(is);
+        throw new FileNotFoundException("Not in classpath: " + getClass().getName().replace('.','/') + FST_FILENAME_SUFFIX);
+      fst = new FST<Long>(new InputStreamDataInput(is), PositiveIntOutputs.getSingleton(true));
     } catch (IOException ioe) {
-      throw new RuntimeException("Cannot load DoubleArrayTrie.", ioe);
+      throw new RuntimeException("Cannot load FST.", ioe);
     } finally {
       IOUtils.closeWhileHandlingException(is);
     }
-    this.trie = trie;
+    // TODO: some way to configure?
+    this.fst = new TokenInfoFST(fst, true);
   }
   
-  public DoubleArrayTrie getTrie() {
-    return trie;
+  public TokenInfoFST getFST() {
+    return fst;
   }
-  
+   
   public synchronized static TokenInfoDictionary getInstance() {
     if (singleton == null) try {
       singleton = new TokenInfoDictionary();

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java?rev=1228810&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoFST.java Sun Jan  8 08:49:12 2012
@@ -0,0 +1,81 @@
+package org.apache.lucene.analysis.kuromoji.dict;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FST.Arc;
+
+public class TokenInfoFST {
+  private final FST<Long> fst;
+
+  // depending upon fasterButMoreRam, we cache root arcs for either 
+  // kana (0x3040-0x30FF) or kana + han (0x3040-0x9FFF)
+  // false: 191 arcs
+  // true:  28,607 arcs (costs ~1.5MB)
+  private final int cacheCeiling;
+  private final FST.Arc<Long> rootCache[];
+  
+  public final Long NO_OUTPUT;
+
+  public TokenInfoFST(FST<Long> fst, boolean fasterButMoreRam) throws IOException {
+    this.fst = fst;
+    this.cacheCeiling = fasterButMoreRam ? 0x9FFF : 0x30FF;
+    NO_OUTPUT = fst.outputs.getNoOutput();
+    rootCache = cacheRootArcs();
+  }
+  
+  @SuppressWarnings("unchecked")
+  private FST.Arc<Long>[] cacheRootArcs() throws IOException {
+    FST.Arc<Long> rootCache[] = new FST.Arc[1+(cacheCeiling-0x3040)];
+    FST.Arc<Long> firstArc = new FST.Arc<Long>();
+    fst.getFirstArc(firstArc);
+    FST.Arc<Long> arc = new FST.Arc<Long>();
+    // TODO: jump to 3040, readNextRealArc to ceiling? (just be careful we don't add bugs)
+    for (int i = 0; i < rootCache.length; i++) {
+      if (fst.findTargetArc(0x3040 + i, firstArc, arc) != null) {
+        rootCache[i] = new FST.Arc<Long>().copyFrom(arc);
+      }
+    }
+    return rootCache;
+  }
+  
+  public FST.Arc<Long> findTargetArc(int ch, FST.Arc<Long> follow, FST.Arc<Long> arc, boolean useCache) throws IOException {
+    if (useCache && ch >= 0x3040 && ch <= cacheCeiling) {
+      assert ch != FST.END_LABEL;
+      final Arc<Long> result = rootCache[ch - 0x3040];
+      if (result == null) {
+        return null;
+      } else {
+        arc.copyFrom(result);
+        return arc;
+      }
+    } else {
+      return fst.findTargetArc(ch, follow, arc);
+    }
+  }
+  
+  public Arc<Long> getFirstArc(FST.Arc<Long> arc) {
+    return fst.getFirstArc(arc);
+  }
+  
+  public Long addOutput(Long prefix, Long output) {
+    return fst.outputs.add(prefix, output);
+  }
+}

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java Sun Jan  8 08:49:12 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -24,14 +25,15 @@ import org.apache.lucene.analysis.kuromo
 import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
 import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
 import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoFST;
 import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
 import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
 import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
+import org.apache.lucene.util.fst.FST;
 
 public class Viterbi {
   
-  private final DoubleArrayTrie trie;
+  private final TokenInfoFST fst;
   
   private final TokenInfoDictionary dictionary;
   
@@ -70,7 +72,7 @@ public class Viterbi {
       UserDictionary userDictionary,
       Mode mode) {
     this.dictionary = dictionary;
-    this.trie = dictionary.getTrie();
+    this.fst = dictionary.getFST();
     this.unkDictionary = unkDictionary;
     this.costs = costs;
     this.userDictionary = userDictionary;
@@ -192,18 +194,20 @@ public class Viterbi {
     
     return result;
   }
-  
-  
+
   /**
    * Build lattice from input text
    * @param text
    */
-  public ViterbiNode[][][] build(char text[], int offset, int length) {
+  public ViterbiNode[][][] build(char text[], int offset, int length) throws IOException {
     ViterbiNode[][] startIndexArr = new ViterbiNode[length + 2][];  // text length + BOS and EOS
     ViterbiNode[][] endIndexArr = new ViterbiNode[length + 2][];  // text length + BOS and EOS
     int[] startSizeArr = new int[length + 2]; // array to keep ViterbiNode count in startIndexArr
     int[] endSizeArr = new int[length + 2];   // array to keep ViterbiNode count in endIndexArr
-    
+    FST.Arc<Long> arc = new FST.Arc<Long>();
+    FST.Arc<Long> endArc = new FST.Arc<Long>();
+    final Long NO_OUTPUT = fst.NO_OUTPUT;
+    Long output;
     ViterbiNode bosNode = new ViterbiNode(0, BOS, 0, BOS.length, 0, 0, 0, -1, Type.KNOWN);
     addToArrays(bosNode, 0, 1, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
     
@@ -224,18 +228,27 @@ public class Viterbi {
       int suffixLength = length - startIndex;
       
       boolean found = false;
+      arc = fst.getFirstArc(arc);
+      output = NO_OUTPUT;
       for (int endIndex = 1; endIndex < suffixLength + 1; endIndex++) {
+        int ch = text[suffixStart + endIndex - 1];
         
-        int result = trie.lookup(text, suffixStart, endIndex);
+        if (fst.findTargetArc(ch, arc, arc, endIndex == 1) == null) {
+          break; // continue to next position
+        } else if (arc.output != NO_OUTPUT) {
+          output = fst.addOutput(output, arc.output);
+        }
         
-        if (result > 0) {	// Found match in double array trie
-          found = true;	// Don't produce unknown word starting from this index
+        if (fst.findTargetArc(FST.END_LABEL, arc, endArc, false) != null) { // Found match in FST
+          int result = endArc.output == NO_OUTPUT 
+              ? output.intValue() 
+              : fst.addOutput(output, endArc.output).intValue();
+          found = true; // Don't produce unknown word starting from this index
+          
           for (int wordId : dictionary.lookupWordIds(result)) {
             ViterbiNode node = new ViterbiNode(wordId, text, suffixStart, endIndex, dictionary.getLeftId(wordId), dictionary.getRightId(wordId), dictionary.getWordCost(wordId), startIndex, Type.KNOWN);
             addToArrays(node, startIndex + 1, startIndex + 1 + endIndex, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
           }
-        } else if(result < 0) {	// If result is less than zero, continue to next position
-          break;						
         }
       }
       

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$fst.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary%24fst.dat?rev=1228810&view=auto
==============================================================================
Binary file - no diff available.

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary$targetMap.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary%24targetMap.dat?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary$targetMap.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary%24targetMap.dat?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/BinaryDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/BinaryDictionaryWriter.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/BinaryDictionaryWriter.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/BinaryDictionaryWriter.java Sun Jan  8 08:49:12 2012
@@ -26,6 +26,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.WritableByteChannel;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.HashMap;
 import java.util.Map;
@@ -39,7 +40,6 @@ import org.apache.lucene.util.RamUsageEs
 import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
 import org.apache.lucene.analysis.kuromoji.dict.BinaryDictionary;
 import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
 
 public abstract class BinaryDictionaryWriter {
   protected final Class<? extends BinaryDictionary> implClazz;
@@ -202,6 +202,7 @@ public abstract class BinaryDictionaryWr
     writePosDict(baseName + BinaryDictionary.POSDICT_FILENAME_SUFFIX);
   }
   
+  // TODO: maybe this int[] should instead be the output to the FST...
   protected void writeTargetMap(String filename) throws IOException {
     new File(filename).getParentFile().mkdirs();
     OutputStream os = new FileOutputStream(filename);
@@ -225,10 +226,13 @@ public abstract class BinaryDictionaryWr
             nulls = 0;
           }
           final int[] a = targetMap[j];
+          Arrays.sort(a, 0, size);
           assert size > 0 && size <= a.length;
           out.writeVInt(size);
+          int prev = 0;
           for (int i = 0; i < size; i++) {
-            out.writeVInt(a[i]);
+            out.writeVInt(a[i] - prev);
+            prev = a[i];
           }
         }
       }

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java Sun Jan  8 08:49:12 2012
@@ -19,9 +19,6 @@ package org.apache.lucene.analysis.kurom
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
 
 public class DictionaryBuilder {
   

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java Sun Jan  8 08:49:12 2012
@@ -27,15 +27,20 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.TreeSet;
 
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoFST;
 import org.apache.lucene.analysis.kuromoji.util.DictionaryBuilder.DictionaryFormat;
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
-import org.apache.lucene.analysis.kuromoji.trie.Trie;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
 
 import com.ibm.icu.text.Normalizer2;
 
@@ -78,19 +83,6 @@ public class TokenInfoDictionaryBuilder 
     Collections.sort(csvFiles);
     return buildDictionary(csvFiles);
   }
-  
-  public DoubleArrayTrie buildDoubleArrayTrie() {
-    Trie tempTrie = buildTrie();
-    return new DoubleArrayTrie(tempTrie);
-  }
-  
-  private Trie buildTrie() {
-    Trie trie = new Trie();
-    for (final String surfaceForm : dictionaryEntries.values()) {
-      trie.add(surfaceForm);
-    }
-    return trie;
-  }
 
   public TokenInfoDictionaryWriter buildDictionary(List<File> csvFiles) throws IOException {
     TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
@@ -138,24 +130,49 @@ public class TokenInfoDictionaryBuilder 
       }
     }
     
-    System.out.print("  building double array trie...");
-    DoubleArrayTrie trie = buildDoubleArrayTrie();
-    dictionary.setTrie(trie);
-    System.out.println("  done");
+    System.out.print(" building FST...");
+    FST<Long> fst = buildFST();
+    dictionary.setFST(fst);
+    System.out.println(" done");
     
     System.out.print("  processing target map...");
-    assert trie != null;
+    TokenInfoFST lookup = new TokenInfoFST(fst, false);
+    assert fst != null;
     for (Entry<Integer, String> entry : entrySet()) {
       int tokenInfoId = entry.getKey();
       String surfaceform = entry.getValue();
-      int doubleArrayId = trie.lookup(surfaceform.toCharArray(), 0, surfaceform.length());
-      assert doubleArrayId > 0;
-      dictionary.addMapping(doubleArrayId, tokenInfoId);
+      int fstId = lookupOrd(lookup, surfaceform);
+      dictionary.addMapping(fstId, tokenInfoId);
     }
+    
     System.out.println("  done");
     
     return dictionary;
   }
+    
+  public int lookupOrd(TokenInfoFST fst, String word) throws IOException {
+    final FST.Arc<Long> arc = fst.getFirstArc(new FST.Arc<Long>());
+    // Accumulate output as we go
+    final Long NO_OUTPUT = fst.NO_OUTPUT;
+    Long output = NO_OUTPUT;
+    for (int i = 0; i < word.length(); i++) {
+      int ch = word.charAt(i);
+      if (fst.findTargetArc(ch, arc, arc, i == 0) == null) {
+        assert false;
+        return -1;
+      } else if (arc.output != NO_OUTPUT) {
+        output = fst.addOutput(output, arc.output);
+      }
+    }
+    if (fst.findTargetArc(FST.END_LABEL, arc, arc, false) == null) {
+      assert false;
+      return -1;
+    } else if (arc.output != NO_OUTPUT) {
+      return fst.addOutput(output, arc.output).intValue();
+    } else {
+      return output.intValue();
+    }
+  }
   
   /*
    * IPADIC features
@@ -212,7 +229,29 @@ public class TokenInfoDictionaryBuilder 
     }
   }
   
-  public Set<Entry<Integer, String>> entrySet() {
+  private Set<Entry<Integer, String>> entrySet() {
     return dictionaryEntries.entrySet();
-  }	
+  }
+  
+  private FST<Long> buildFST() throws IOException {    
+    FST<Long> words;
+    Collection<String> values = dictionaryEntries.values();
+    TreeSet<String> unique = new TreeSet<String>(values);
+    PositiveIntOutputs o = PositiveIntOutputs.getSingleton(true);
+    Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE2, o);
+    IntsRef scratch = new IntsRef();
+    long ord = 1;
+    for (String entry : unique) {
+      scratch.grow(entry.length());
+      scratch.length = entry.length();
+      for (int i = 0; i < entry.length(); i++) {
+        scratch.ints[i] = (int) entry.charAt(i);
+      }
+      b.add(scratch, ord);
+      ord++;
+    }
+    words = b.finish();
+    System.out.print(" " + words.getNodeCount() + " nodes, " + words.getArcCount() + " arcs, " + words.sizeInBytes() + " bytes...  ");
+    return words;
+  }
 }

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java?rev=1228810&r1=1228809&r2=1228810&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java Sun Jan  8 08:49:12 2012
@@ -18,39 +18,31 @@ package org.apache.lucene.analysis.kurom
  */
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
-import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
+import org.apache.lucene.util.fst.FST;
 
 public class TokenInfoDictionaryWriter extends BinaryDictionaryWriter {
-
-  private DoubleArrayTrie trie;
+  private FST<Long> fst;
 
   public TokenInfoDictionaryWriter(int size) {
     super(TokenInfoDictionary.class, size);
-    this.trie = trie;
   }
   
-  public void setTrie(DoubleArrayTrie trie) {
-    this.trie = trie;
+  public void setFST(FST<Long> fst) {
+    this.fst = fst;
   }
   
   @Override
   public void write(String baseDir) throws IOException {
     super.write(baseDir);
-    writeDoubleArrayTrie(getBaseFileName(baseDir) + TokenInfoDictionary.TRIE_FILENAME_SUFFIX);
-  }
-  
-  protected void writeDoubleArrayTrie(String filename) throws IOException  {
-    new File(filename).getParentFile().mkdirs();
-    final FileOutputStream os = new FileOutputStream(filename);
-    try {
-      trie.write(os);
-    } finally {
-      os.close();
-    }
+    writeFST(getBaseFileName(baseDir) + TokenInfoDictionary.FST_FILENAME_SUFFIX);
   }
   
+  protected void writeFST(String filename) throws IOException {
+    File f = new File(filename);
+    f.getParentFile().mkdirs();
+    fst.save(f);
+  }  
 }