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 2013/10/21 14:04:43 UTC

svn commit: r1534115 - in /lucene/dev/branches/lucene4956/lucene/analysis/arirang/src: java/org/apache/lucene/analysis/ko/dic/ resources/org/apache/lucene/analysis/ko/dic/ tools/java/org/apache/lucene/analysis/ko/dic/

Author: rmuir
Date: Mon Oct 21 12:04:43 2013
New Revision: 1534115

URL: http://svn.apache.org/r1534115
Log:
LUCENE-4956: commit working state

Added:
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/ByteOutputs.java   (with props)
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/HangulDictionary.java   (with props)
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/words.dat   (with props)
Modified:
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryResources.java
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryUtil.java
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/WordEntry.java
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/extension.dic
    lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/tools/java/org/apache/lucene/analysis/ko/dic/DictionaryBuilder.java

Added: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/ByteOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/ByteOutputs.java?rev=1534115&view=auto
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/ByteOutputs.java (added)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/ByteOutputs.java Mon Oct 21 12:04:43 2013
@@ -0,0 +1,99 @@
+package org.apache.lucene.analysis.ko.dic;
+
+/**
+ * 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.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.fst.Outputs;
+
+/**
+ * Output is a byte, for each input term.
+ * doesn't share anything.
+ *
+ * @lucene.experimental
+ */
+final class ByteOutputs extends Outputs<Byte> {
+  private final static Byte NO_OUTPUT = new Byte((byte)0);
+  private final static ByteOutputs singleton = new ByteOutputs();
+
+  public static ByteOutputs getSingleton() {
+    return singleton;
+  }
+
+  @Override
+  public Byte common(Byte output1, Byte output2) {
+    assert valid(output1);
+    assert valid(output2);
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public Byte subtract(Byte output, Byte inc) {
+    assert valid(output);
+    assert valid(inc);
+    assert inc == NO_OUTPUT;
+    return output;
+  }
+
+  @Override
+  public Byte add(Byte prefix, Byte output) {
+    assert valid(prefix);
+    assert valid(output);
+    assert output == NO_OUTPUT || prefix == NO_OUTPUT;
+    
+    if (prefix == NO_OUTPUT) {
+      return output;
+    } else {
+      return prefix;
+    }
+  }
+
+  @Override
+  public void write(Byte output, DataOutput out) throws IOException {
+    assert valid(output);
+    assert output != 0;
+    out.writeByte(output);
+  }
+
+  @Override
+  public Byte read(DataInput in) throws IOException {
+    byte v = in.readByte();
+    assert v != 0;
+    return v;
+  }
+
+  private boolean valid(Byte o) {
+    assert o != null;
+    assert o instanceof Byte;
+    assert o == NO_OUTPUT || o != 0;
+    return true;
+  }
+
+  @Override
+  public Byte getNoOutput() {
+    return NO_OUTPUT;
+  }
+
+  @Override
+  public String outputToString(Byte output) {
+    return output.toString();
+  }
+}
+

Modified: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryResources.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryResources.java?rev=1534115&r1=1534114&r2=1534115&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryResources.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryResources.java Mon Oct 21 12:04:43 2013
@@ -49,6 +49,7 @@ public class DictionaryResources {
   public static final String FILE_SYLLABLE_DAT = "syllable.dat";
   public static final String FILE_HANJA_IDX = "hanja.idx";
   public static final String FILE_HANJA_DAT = "hanja.dat";
+  public static final String FILE_WORDS_DAT = "words.dat";
   public static final int DATA_VERSION = 0;
 
   private DictionaryResources() {}

Modified: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryUtil.java?rev=1534115&r1=1534114&r2=1534115&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryUtil.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/DictionaryUtil.java Mon Oct 21 12:04:43 2013
@@ -17,7 +17,9 @@ package org.apache.lucene.analysis.ko.di
  * limitations under the License.
  */
 
+import java.io.BufferedInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -25,12 +27,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.ko.utils.Trie;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.fst.FST;
 
 public class DictionaryUtil {
   private DictionaryUtil() {}
   
   private static final Trie<String,WordEntry> dictionary = new Trie<String, WordEntry>(false);
   
+  private static final HangulDictionary newDictionary;
+  
   private static final Set<String> josas = new HashSet<String>();
   
   private static final Set<String> eomis = new HashSet<String>();;
@@ -97,6 +105,15 @@ public class DictionaryUtil {
   
       readFileToSet(suffixs,DictionaryResources.FILE_SUFFIX);
       
+      InputStream stream = DictionaryResources.class.getResourceAsStream(DictionaryResources.FILE_WORDS_DAT);
+      DataInput dat = new InputStreamDataInput(new BufferedInputStream(stream));
+      CodecUtil.checkHeader(dat, DictionaryResources.FILE_WORDS_DAT, DictionaryResources.DATA_VERSION, DictionaryResources.DATA_VERSION);
+      byte metadata[] = new byte[dat.readByte() * HangulDictionary.RECORD_SIZE];
+      dat.readBytes(metadata, 0, metadata.length);
+      ByteOutputs outputs = ByteOutputs.getSingleton();
+      FST<Byte> fst = new FST<Byte>(dat, outputs);
+      newDictionary = new HangulDictionary(fst, metadata);
+      stream.close();
     } catch (IOException e) {
       throw new Error("Cannot load resource",e);
     }
@@ -107,25 +124,37 @@ public class DictionaryUtil {
     return dictionary.getPrefixedBy(prefix);
   }
 
+  /** only use this if you surely need the whole entry */
   public static WordEntry getWord(String key) {    
-    if(key.length()==0) return null;
-    
-    return (WordEntry)dictionary.get(key);
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    } else {
+      return newDictionary.decodeEntry(key, b);
+    }
   }
   
   public static WordEntry getWordExceptVerb(String key) {
-    WordEntry entry = getWord(key);
-    if (entry != null && (entry.isNoun() || entry.isAdverb())) {
-      return entry;
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    }
+    char flags = newDictionary.getFlags(b);
+    if ((flags & (WordEntry.NOUN | WordEntry.BUSA)) != 0) {
+      return newDictionary.decodeEntry(key, b, flags);
     } else {
       return null;
     }
   }
   
-  public static WordEntry getNoun(String key) {  
-    WordEntry entry = getWord(key);
-    if (entry != null && entry.isNoun() && !entry.isCompoundNoun()) {
-      return entry;
+  public static WordEntry getNoun(String key) {
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    }
+    char flags = newDictionary.getFlags(b);
+    if ((flags & WordEntry.NOUN) != 0 && (flags & WordEntry.COMPOUND) == 0) {
+      return newDictionary.decodeEntry(key, b, flags);
     } else {
       return null;
     }
@@ -138,27 +167,39 @@ public class DictionaryUtil {
    * @return  WordEntry
    */
   public static WordEntry getAllNoun(String key) {  
-    WordEntry entry = getWord(key);
-    if (entry != null && entry.isNoun()) {
-      return entry;
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    }
+    char flags = newDictionary.getFlags(b);
+    if ((flags & WordEntry.NOUN) != 0) {
+      return newDictionary.decodeEntry(key, b, flags);
     } else {
       return null;
     }
   }
   
   public static WordEntry getVerb(String key) {
-    WordEntry entry = getWord(key);  
-    if (entry != null && entry.isVerb()) {
-      return entry;
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    }
+    char flags = newDictionary.getFlags(b);
+    if ((flags & WordEntry.VERB) != 0) {
+      return newDictionary.decodeEntry(key, b, flags);
     } else {
       return null;
     }
   }
   
   public static WordEntry getBusa(String key) {
-    WordEntry entry = getWord(key);
-    if (entry != null && entry.isAdverb() && !entry.isNoun()) {
-      return entry;
+    Byte b = newDictionary.lookup(key);
+    if (b == null) {
+      return null;
+    }
+    char flags = newDictionary.getFlags(b);
+    if ((flags & WordEntry.BUSA) != 0 && (flags & WordEntry.NOUN) == 0) {
+      return newDictionary.decodeEntry(key, b, flags);
     } else {
       return null;
     }

Added: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/HangulDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/HangulDictionary.java?rev=1534115&view=auto
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/HangulDictionary.java (added)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/HangulDictionary.java Mon Oct 21 12:04:43 2013
@@ -0,0 +1,121 @@
+package org.apache.lucene.analysis.ko.dic;
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FST.BytesReader;
+
+class HangulDictionary {
+  final FST<Byte> fst;
+  final byte[] metadata;
+  
+  static final int RECORD_SIZE = 15;
+  
+  public HangulDictionary(FST<Byte> fst, byte[] metadata) {
+    this.fst = fst;
+    this.metadata = metadata;
+  }
+  
+  Byte lookup(String key) {
+    // TODO: why is does this thing lookup empty strings?
+    if (key.length() == 0) {
+      return null;
+    }
+    final FST.Arc<Byte> arc = fst.getFirstArc(new FST.Arc<Byte>());
+
+    final BytesReader fstReader = fst.getBytesReader();
+
+    // Accumulate output as we go
+    Byte output = fst.outputs.getNoOutput();
+    for (int i = 0; i < key.length(); i++) {
+      try {
+        if (fst.findTargetArc(key.charAt(i), arc, arc, fstReader) == null) {
+          return null;
+        }
+      } catch (IOException bogus) {
+        throw new RuntimeException();
+      }
+      // we shouldnt need this accumulation?!
+      output = fst.outputs.add(output, arc.output);
+    }
+
+    if (arc.isFinal()) {
+      return fst.outputs.add(output, arc.nextFinalOutput);
+    } else {
+      return null;
+    }
+  }
+  
+  char getFlags(byte b) {
+    int off = b * RECORD_SIZE;
+    return (char)((metadata[off] << 8) | (metadata[off+1] & 0xff));
+  }
+  
+  WordEntry decodeEntry(String key, byte b) {
+    return decodeEntry(key, b, getFlags(b));
+  }
+  
+  WordEntry decodeEntry(String key, byte b, char flags) {
+    if ((flags & WordEntry.COMPOUND_IRREGULAR) != 0) {
+      return new WordEntry(key, flags, getIrregularCompounds(key, b));
+    } else if ((flags & WordEntry.COMPOUND) != 0) {
+      return new WordEntry(key, flags, getCompounds(key, b));
+    } else {
+      return new WordEntry(key, flags, null);
+    }
+  }
+  
+  List<CompoundEntry> getCompounds(String word, byte b) {
+    int off = b * RECORD_SIZE;
+    int numSplits = metadata[off+2];
+    assert numSplits > 0;
+    List<CompoundEntry> compounds = new ArrayList<>(numSplits+1);
+    int last = 0;
+    for (int i = 0; i < numSplits; i++) {
+      int split = metadata[off+3+i];
+      compounds.add(new CompoundEntry(word.substring(last, split), true));
+      last = split;
+    }
+    compounds.add(new CompoundEntry(word.substring(last), true));
+    return compounds;
+  }
+  
+  List<CompoundEntry> getIrregularCompounds(String word, byte b) {
+    int off = b * RECORD_SIZE;
+    int numChars = metadata[off+2];
+    // TODO: more efficient
+    List<CompoundEntry> compounds = new ArrayList<>();
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < numChars; i++) {
+      int idx = off+3+(i<<1);
+      char next = (char)((metadata[idx] << 8) | (metadata[idx+1] & 0xff));
+      if (next == ',') {
+        compounds.add(new CompoundEntry(sb.toString(), true));
+        sb.setLength(0);
+      } else {
+        sb.append(next);
+      }
+    }
+    compounds.add(new CompoundEntry(sb.toString(), true));
+    return compounds;
+  }
+}

Modified: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/WordEntry.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/WordEntry.java?rev=1534115&r1=1534114&r2=1534115&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/WordEntry.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/java/org/apache/lucene/analysis/ko/dic/WordEntry.java Mon Oct 21 12:04:43 2013
@@ -29,6 +29,7 @@ public class WordEntry {
   static final int BEV =      1 << 7;
   static final int NE  =      1 << 8;
   static final int COMPOUND = 1 << 9;
+  static final int COMPOUND_IRREGULAR = 1 << 10;
   
   /** Regular verb type */
   public static final int VERB_TYPE_REGULAR = 0;
@@ -62,12 +63,12 @@ public class WordEntry {
   /**
    * 단어특성
    */
-  private final char features;
+  final char features;
   
-  private final List<CompoundEntry> compounds;
+  final List<CompoundEntry> compounds;
   
   public WordEntry(String word, int features, List<CompoundEntry> compounds) {
-    if (features < 0 || features >= 1024) {
+    if (features < 0 || features >= 2048) {
       throw new IllegalArgumentException("Invalid features: " + Integer.toHexString(features));
     }
     this.word = word;

Modified: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/extension.dic
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/extension.dic?rev=1534115&r1=1534114&r2=1534115&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/extension.dic (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/extension.dic Mon Oct 21 12:04:43 2013
@@ -5339,7 +5339,7 @@
 히포크라테스,100110000X
 히피스트럼,100110000X
 힌지,100110000X
-! compound exceptions
+! moved from compounds.dic, previously decompounded to themselves
 경영인,100000000X
 경영자,100000000X
 경영주,100000000X

Added: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/words.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/resources/org/apache/lucene/analysis/ko/dic/words.dat?rev=1534115&view=auto
==============================================================================
Binary file - no diff available.

Modified: lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/tools/java/org/apache/lucene/analysis/ko/dic/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/tools/java/org/apache/lucene/analysis/ko/dic/DictionaryBuilder.java?rev=1534115&r1=1534114&r2=1534115&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/tools/java/org/apache/lucene/analysis/ko/dic/DictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene4956/lucene/analysis/arirang/src/tools/java/org/apache/lucene/analysis/ko/dic/DictionaryBuilder.java Mon Oct 21 12:04:43 2013
@@ -23,8 +23,15 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataOutput;
@@ -32,6 +39,10 @@ import org.apache.lucene.store.OutputStr
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
+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.Outputs;
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -57,6 +68,7 @@ public class DictionaryBuilder {
     }
     buildHanjaMap(inputDir, outputDir);
     buildSyllableDict(inputDir, outputDir);
+    buildHangulDict(inputDir, outputDir);
   }
   
   static void copyAsIs(File in, File out) throws Exception {
@@ -213,4 +225,248 @@ public class DictionaryBuilder {
     reader.close();
     stream.close();
   }
+  
+  
+  static void buildHangulDict(File inputDir, File outputDir) throws Exception {
+    TreeMap<String,Integer> sorted = new TreeMap<String,Integer>();
+    Map<Output,Integer> classes = new LinkedHashMap<>();
+    File input = new File(inputDir, "dictionary.dic");
+    BufferedReader reader = new BufferedReader(IOUtils.getDecodingReader(input, IOUtils.CHARSET_UTF_8));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("!") && !line.startsWith("\uFEFF")) {
+        processLine(line, sorted, classes);
+      }
+    }
+    reader.close();
+    input = new File(inputDir, "extension.dic");
+    reader = new BufferedReader(IOUtils.getDecodingReader(input, IOUtils.CHARSET_UTF_8));
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("!") && !line.startsWith("\uFEFF")) {
+        processLine(line, sorted, classes);
+      }
+    }
+    reader.close();
+    input = new File(inputDir, "compounds.dic");
+    reader = new BufferedReader(IOUtils.getDecodingReader(input, IOUtils.CHARSET_UTF_8));
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("!") && !line.startsWith("\uFEFF")) {
+        processCompound(line, sorted, classes);
+      }
+    }
+    reader.close();
+    System.out.println("#words: " + sorted.size());
+    System.out.println("#classes: " + classes.size());
+    Outputs<Byte> fstOutput = ByteOutputs.getSingleton();
+    //    makes corrupt FST!!!!
+    //     Builder<Byte> builder = new Builder<Byte>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, false, PackedInts.DEFAULT, true, 15);
+    Builder<Byte> builder = new Builder<Byte>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true, PackedInts.DEFAULT, true, 15);
+    IntsRef scratch = new IntsRef();
+    for (Map.Entry<String,Integer> e : sorted.entrySet()) {
+      String token = e.getKey();
+      scratch.grow(token.length());
+      scratch.length = token.length();
+      for (int i = 0; i < token.length(); i++) {
+        scratch.ints[i] = (int) token.charAt(i);
+      }
+      int v = e.getValue();
+      assert v >= 0 && v < 128;
+      builder.add(scratch, (byte)v);
+    }
+    FST<Byte> fst = builder.finish();
+    System.out.println("FST size: " + fst.sizeInBytes());
+    OutputStream stream = new BufferedOutputStream(new FileOutputStream(new File(outputDir, DictionaryResources.FILE_WORDS_DAT)));
+    DataOutput out = new OutputStreamDataOutput(stream);
+    CodecUtil.writeHeader(out, DictionaryResources.FILE_WORDS_DAT, DictionaryResources.DATA_VERSION);
+    out.writeByte((byte)classes.size());
+    for (Output o : classes.keySet()) {
+      o.write(out);
+    }
+    fst.save(out);
+  }
+  
+  static void processLine(String line, TreeMap<String,Integer> sorted, Map<Output,Integer> classes) {
+    String[] infos = line.split("[,]+");
+    assert infos.length == 2;
+    assert infos[1].length() == 10;
+    Output output = new Output();
+    output.flags = (char) parseFlags(infos[1]);
+    output.splits = Collections.emptyList();
+    Integer ord = classes.get(output);
+    if (ord == null) {
+      ord = classes.size();
+      classes.put(output, ord);
+    }
+    sorted.put(infos[0], ord);
+  }
+  
+  static void processCompound(String line, TreeMap<String,Integer> sorted, Map<Output,Integer> classes) {
+    String[] infos = line.split("[:]+");
+    assert infos.length == 3;
+    assert infos[2].length() == 4;
+    Output output = new Output();
+    
+    if (!infos[1].replace(",", "").equals(infos[0])) {
+      output.flags = (char) parseFlags("300"+infos[2]+"00X");
+      output.decomp = infos[1];
+    } else {
+      output.flags = (char) parseFlags("200"+infos[2]+"00X");
+      output.splits = parseSplits(infos[1]);
+    }
+    
+    Integer ord = classes.get(output);
+    if (ord == null) {
+      ord = classes.size();
+      classes.put(output, ord);
+    }
+    sorted.put(infos[0], ord);
+  }
+  
+  static List<Integer> parseSplits(String line) {
+    List<Integer> splits = new ArrayList<>();
+    int current = 0;
+    while (true) {
+      current = line.indexOf(',', current);
+      assert current != 0;
+      assert current != line.length();
+      if (current < 0) {
+        break;
+      }
+      splits.add(current - splits.size());
+      current++;
+    }
+    
+    // validate splits data
+    assert !splits.isEmpty();
+    String comp = line.replaceAll(",", "");
+    StringBuilder sb = new StringBuilder();
+    int last = 0;
+    for (int i : splits) {
+      assert i < comp.length();
+      assert i > last;
+      sb.append(comp.substring(last, i));
+      last = i;
+    }
+    sb.append(comp.substring(last));
+    assert sb.toString().equals(comp);
+    
+    return splits;
+  }
+    
+  
+  static class Output {
+    char flags;
+    List<Integer> splits;
+    String decomp;
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((decomp == null) ? 0 : decomp.hashCode());
+      result = prime * result + flags;
+      result = prime * result + ((splits == null) ? 0 : splits.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      Output other = (Output) obj;
+      if (decomp == null) {
+        if (other.decomp != null) return false;
+      } else if (!decomp.equals(other.decomp)) return false;
+      if (flags != other.flags) return false;
+      if (splits == null) {
+        if (other.splits != null) return false;
+      } else if (!splits.equals(other.splits)) return false;
+      return true;
+    }
+
+    static final int MAX_SPLITS = HangulDictionary.RECORD_SIZE - 3;
+    
+    public void write(DataOutput output) throws IOException {
+      output.writeShort((short)flags);
+      if (decomp != null) {
+        assert decomp.length() <= MAX_SPLITS/2;
+        output.writeByte((byte) decomp.length());
+        for (int i = 0; i < decomp.length(); i++) {
+          output.writeShort((short) decomp.charAt(i));
+        }
+        for (int i = decomp.length(); i < MAX_SPLITS/2; i++) {
+          output.writeShort((short)0);
+        }
+      } else {
+        assert splits.size() <= MAX_SPLITS;
+        output.writeByte((byte) splits.size());
+        for (int i : splits) {
+          output.writeByte((byte)i);
+        }
+        for (int i = splits.size(); i < MAX_SPLITS; i++) {
+          output.writeByte((byte)0);
+        }
+      }
+    }
+  }
+  
+  private static int parseFlags(String buffer) {
+    if (buffer.length() != 10) {
+      throw new IllegalArgumentException("Invalid flags: " + buffer);
+    }
+    int flags = 0;
+    // IDX_NOUN: 1 if noun, 2 if compound, 3 if "strange compound"
+    if (buffer.charAt(0) == '3') {
+      flags |= WordEntry.COMPOUND | WordEntry.COMPOUND_IRREGULAR | WordEntry.NOUN;
+    } else if (buffer.charAt(0) == '2') {
+      flags |= WordEntry.COMPOUND | WordEntry.NOUN;
+    } else if (buffer.charAt(0) == '1') {
+      flags |= WordEntry.NOUN;
+    } else if (buffer.charAt(0) != '0') {
+      throw new IllegalArgumentException("Invalid flags: " + buffer);
+    }
+    // IDX_VERB
+    if (parseBoolean(buffer, 1)) {
+      flags |= WordEntry.VERB;
+    }
+    // IDX_BUSA
+    if (parseBoolean(buffer, 2)) {
+      flags |= WordEntry.BUSA;
+    }
+    // IDX_DOV
+    if (parseBoolean(buffer, 3)) {
+      flags |= WordEntry.DOV;
+    }
+    // IDX_BEV
+    if (parseBoolean(buffer, 4)) {
+      flags |= WordEntry.BEV;
+    }
+    // IDX_NE
+    if (parseBoolean(buffer, 5)) {
+      flags |= WordEntry.NE;
+    }
+    // IDX_REGURA
+    switch(buffer.charAt(9)) {
+      case 'B': return flags | WordEntry.VERB_TYPE_BIUP;
+      case 'H': return flags | WordEntry.VERB_TYPE_HIOOT;
+      case 'U': return flags | WordEntry.VERB_TYPE_LIUL;
+      case 'L': return flags | WordEntry.VERB_TYPE_LOO;
+      case 'S': return flags | WordEntry.VERB_TYPE_SIUT;
+      case 'D': return flags | WordEntry.VERB_TYPE_DI;
+      case 'R': return flags | WordEntry.VERB_TYPE_RU;
+      case 'X': return flags | WordEntry.VERB_TYPE_REGULAR;
+      default: throw new IllegalArgumentException("Invalid flags: " + buffer);
+    }
+  }
+  
+  private static boolean parseBoolean(String buffer, int position) {
+    if (buffer.charAt(position) == '1') {
+      return true;
+    } else if (buffer.charAt(position) == '0') {
+      return false;
+    } else {
+      throw new IllegalArgumentException("Invalid flags: " + buffer);
+    }
+  }
 }