You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ji...@apache.org on 2018/04/13 12:17:57 UTC

[2/4] lucene-solr:branch_7x: LUCENE-8231: Add a new analysis module (nori) similar to Kuromoji but to handle Korean

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanAnalyzer.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanAnalyzer.java
new file mode 100644
index 0000000..fd574ce
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanAnalyzer.java
@@ -0,0 +1,109 @@
+/*
+ * 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.ko;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+
+/**
+ * Test Korean morphological analyzer
+ */
+public class TestKoreanAnalyzer extends BaseTokenStreamTestCase {
+  public void testSentence() throws IOException {
+    Analyzer a = new KoreanAnalyzer();
+    assertAnalyzesTo(a, "한국은 대단한 나라입니다.",
+        new String[]{"한국", "대단", "나라", "이"},
+        new int[]{ 0, 4, 8, 10 },
+        new int[]{ 2, 6, 10, 13 },
+        new int[]{ 1, 2, 3, 1 }
+    );
+    a.close();
+  }
+
+  public void testStopTags() throws IOException {
+    Set<POS.Tag> stopTags = Arrays.asList(POS.Tag.NNP, POS.Tag.NNG).stream().collect(Collectors.toSet());
+    Analyzer a = new KoreanAnalyzer(null, KoreanTokenizer.DecompoundMode.DISCARD, stopTags, false);
+    assertAnalyzesTo(a, "한국은 대단한 나라입니다.",
+        new String[]{"은", "대단", "하", "ᆫ", "이", "ᄇ니다"},
+        new int[]{ 2, 4, 6, 6, 10, 10 },
+        new int[]{ 3, 6, 7, 7, 13, 13 },
+        new int[]{ 2, 1, 1, 1, 2, 1 }
+    );
+    a.close();
+  }
+
+  public void testUnknownWord() throws IOException {
+    Analyzer a = new KoreanAnalyzer(null, KoreanTokenizer.DecompoundMode.DISCARD,
+        KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS, true);
+
+    assertAnalyzesTo(a,"2018 평창 동계올림픽대회",
+        new String[]{"2", "0", "1", "8", "평창", "동계", "올림픽", "대회"},
+        new int[]{0, 1, 2, 3, 5, 8, 10, 13},
+        new int[]{1, 2, 3, 4, 7, 10, 13, 15},
+        new int[]{1, 1, 1, 1, 1, 1, 1, 1});
+    a.close();
+
+    a = new KoreanAnalyzer(null, KoreanTokenizer.DecompoundMode.DISCARD,
+        KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS, false);
+
+    assertAnalyzesTo(a,"2018 평창 동계올림픽대회",
+        new String[]{"2018", "평창", "동계", "올림픽", "대회"},
+        new int[]{0, 5, 8, 10, 13},
+        new int[]{4, 7, 10, 13, 15},
+        new int[]{1, 1, 1, 1, 1});
+    a.close();
+  }
+
+  /**
+   * blast random strings against the analyzer
+   */
+  public void testRandom() throws IOException {
+    Random random = random();
+    final Analyzer a = new KoreanAnalyzer();
+    checkRandomData(random, a, atLeast(1000));
+    a.close();
+  }
+
+  /**
+   * blast some random large strings through the analyzer
+   */
+  public void testRandomHugeStrings() throws Exception {
+    Random random = random();
+    final Analyzer a = new KoreanAnalyzer();
+    checkRandomData(random, a, 2 * RANDOM_MULTIPLIER, 8192);
+    a.close();
+  }
+
+  // Copied from TestKoreanTokenizer, to make sure passing
+  // user dict to analyzer works:
+  public void testUserDict() throws IOException {
+    final Analyzer analyzer = new KoreanAnalyzer(TestKoreanTokenizer.readDict(),
+        KoreanTokenizer.DEFAULT_DECOMPOUND, KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS, false);
+    assertAnalyzesTo(analyzer, "c++ 프로그래밍 언어",
+        new String[]{"c++", "프로그래밍", "언어"},
+        new int[]{0, 4, 10},
+        new int[]{3, 9, 12},
+        new int[]{1, 1, 1}
+    );
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanPartOfSpeechStopFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanPartOfSpeechStopFilterFactory.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanPartOfSpeechStopFilterFactory.java
new file mode 100644
index 0000000..5486f3f
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanPartOfSpeechStopFilterFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ko;
+
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.util.Version;
+
+/**
+ * Simple tests for {@link KoreanPartOfSpeechStopFilterFactory}
+ */
+public class TestKoreanPartOfSpeechStopFilterFactory extends BaseTokenStreamTestCase {
+  public void testStopTags() throws IOException {
+    KoreanTokenizerFactory tokenizerFactory = new KoreanTokenizerFactory(new HashMap<String,String>());
+    tokenizerFactory.inform(new StringMockResourceLoader(""));
+    TokenStream ts = tokenizerFactory.create();
+    ((Tokenizer)ts).setReader(new StringReader(" 한국은 대단한 나라입니다."));
+    Map<String,String> args = new HashMap<>();
+    args.put("luceneMatchVersion", Version.LATEST.toString());
+    args.put("tags", "E, J");
+    KoreanPartOfSpeechStopFilterFactory factory = new KoreanPartOfSpeechStopFilterFactory(args);
+    ts = factory.create(ts);
+    assertTokenStreamContents(ts,
+        new String[] { "한국", "대단", "하", "나라", "이" }
+    );
+  }
+
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      new KoreanPartOfSpeechStopFilterFactory(new HashMap<String,String>() {{
+        put("luceneMatchVersion", Version.LATEST.toString());
+        put("bogusArg", "bogusValue");
+      }});
+    });
+    assertTrue(expected.getMessage().contains("Unknown parameters"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilter.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilter.java
new file mode 100644
index 0000000..0c5a40c
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilter.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ko;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Tests for {@link TestKoreanReadingFormFilter}
+ */
+public class TestKoreanReadingFormFilter extends BaseTokenStreamTestCase {
+  private Analyzer analyzer;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer =
+            new KoreanTokenizer(newAttributeFactory(), null, KoreanTokenizer.DecompoundMode.DISCARD, false);
+        return new TokenStreamComponents(tokenizer, new KoreanReadingFormFilter(tokenizer));
+      }
+    };
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    IOUtils.close(analyzer);
+    super.tearDown();
+  }
+
+  public void testReadings() throws IOException {
+    assertAnalyzesTo(analyzer, "車丞相",
+        new String[] { "차", "승상" }
+    );
+  }
+  
+  public void testRandomData() throws IOException {
+    Random random = random();
+    checkRandomData(random, analyzer, 1000*RANDOM_MULTIPLIER);
+  }
+  
+  public void testEmptyTerm() throws IOException {
+    Analyzer a = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KeywordTokenizer();
+        return new TokenStreamComponents(tokenizer, new KoreanReadingFormFilter(tokenizer));
+      }
+    };
+    checkOneTerm(a, "", "");
+    a.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilterFactory.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilterFactory.java
new file mode 100644
index 0000000..f058a44
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanReadingFormFilterFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.ko;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+
+/**
+ * Simple tests for {@link KoreanReadingFormFilterFactory}
+ */
+public class TestKoreanReadingFormFilterFactory extends BaseTokenStreamTestCase {
+  public void testReadings() throws IOException {
+    KoreanTokenizerFactory tokenizerFactory = new KoreanTokenizerFactory(new HashMap<>());
+    tokenizerFactory.inform(new StringMockResourceLoader(""));
+    TokenStream tokenStream = tokenizerFactory.create();
+    ((Tokenizer)tokenStream).setReader(new StringReader("丞相"));
+    KoreanReadingFormFilterFactory filterFactory = new KoreanReadingFormFilterFactory(new HashMap<>());
+    assertTokenStreamContents(filterFactory.create(tokenStream),
+        new String[] { "승상" }
+    );
+  }
+  
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      new KoreanReadingFormFilterFactory(new HashMap<String,String>() {{
+        put("bogusArg", "bogusValue");
+      }});
+    });
+    assertTrue(expected.getMessage().contains("Unknown parameters"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizer.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizer.java
new file mode 100644
index 0000000..0471e5f
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizer.java
@@ -0,0 +1,355 @@
+/*
+ * 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.ko;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Random;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockGraphTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.ko.KoreanTokenizer.DecompoundMode;
+import org.apache.lucene.analysis.ko.dict.UserDictionary;
+import org.apache.lucene.analysis.ko.tokenattributes.PartOfSpeechAttribute;
+import org.apache.lucene.analysis.ko.tokenattributes.ReadingAttribute;
+
+public class TestKoreanTokenizer extends BaseTokenStreamTestCase {
+  private Analyzer analyzer, analyzerUnigram, analyzerDecompound, analyzerDecompoundKeep, analyzerReading;
+
+  public static UserDictionary readDict() {
+    InputStream is = TestKoreanTokenizer.class.getResourceAsStream("userdict.txt");
+    if (is == null) {
+      throw new RuntimeException("Cannot find userdict.txt in test classpath!");
+    }
+    try {
+      try {
+        Reader reader = new InputStreamReader(is, StandardCharsets.UTF_8);
+        return UserDictionary.open(reader);
+      } finally {
+        is.close();
+      }
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe);
+    }
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    UserDictionary userDictionary = readDict();
+    analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), userDictionary,
+            DecompoundMode.NONE, false);
+        return new TokenStreamComponents(tokenizer, tokenizer);
+      }
+    };
+    analyzerUnigram = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), userDictionary,
+            DecompoundMode.NONE, true);
+        return new TokenStreamComponents(tokenizer, tokenizer);
+      }
+    };
+    analyzerDecompound = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), userDictionary,
+            DecompoundMode.DISCARD, false);
+        return new TokenStreamComponents(tokenizer);
+      }
+    };
+    analyzerDecompoundKeep = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), userDictionary,
+            DecompoundMode.MIXED, false);
+        return new TokenStreamComponents(tokenizer);
+      }
+    };
+    analyzerReading = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), userDictionary,
+            DecompoundMode.NONE, false);
+        KoreanReadingFormFilter filter = new KoreanReadingFormFilter(tokenizer);
+        return new TokenStreamComponents(tokenizer, filter);
+      }
+    };
+  }
+
+  public void testSpaces() throws IOException {
+    assertAnalyzesTo(analyzer, "화학        이외의         것",
+        new String[]{"화학", "이외", "의", "것"},
+        new int[]{0, 10, 12, 22},
+        new int[]{2, 12, 13, 23},
+        new int[]{1, 1, 1, 1}
+    );
+    assertPartsOfSpeech(analyzer, "화학 이외의         것",
+        new POS.Type[] { POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME },
+        new POS.Tag[] { POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNB },
+        new POS.Tag[] { POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNB }
+    );
+  }
+
+  public void testPartOfSpeechs() throws IOException {
+    assertAnalyzesTo(analyzer, "화학 이외의 것",
+        new String[]{"화학", "이외", "의", "것"},
+        new int[]{0, 3, 5, 7},
+        new int[]{2, 5, 6, 8},
+        new int[]{1, 1, 1, 1}
+    );
+    assertPartsOfSpeech(analyzer, "화학 이외의 것",
+        new POS.Type[] { POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME },
+        new POS.Tag[] { POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNB },
+        new POS.Tag[] { POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNB }
+    );
+  }
+
+  public void testPartOfSpeechsWithCompound() throws IOException {
+    assertAnalyzesTo(analyzer, "가락지나물은 한국, 중국, 일본",
+        new String[]{"가락지나물", "은", "한국", "중국", "일본"},
+        new int[]{0, 5, 7, 11, 15},
+        new int[]{5, 6, 9, 13, 17},
+        new int[]{1, 1, 1, 1, 1}
+    );
+
+    assertPartsOfSpeech(analyzer,"가락지나물은 한국, 중국, 일본",
+        new POS.Type[]{POS.Type.COMPOUND, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP}
+    );
+
+    assertAnalyzesTo(analyzerDecompound, "가락지나물은 한국, 중국, 일본",
+        new String[]{"가락지", "나물", "은", "한국", "중국", "일본"},
+        new int[]{0, 3, 5, 7, 11, 15},
+        new int[]{3, 5, 6, 9, 13, 17},
+        new int[]{1, 1, 1, 1, 1, 1}
+    );
+
+    assertAnalyzesTo(analyzerDecompoundKeep, "가락지나물은 한국, 중국, 일본",
+        new String[]{"가락지나물", "가락지", "나물", "은", "한국", "중국", "일본"},
+        new int[]{0, 0, 3, 5, 7, 11, 15},
+        new int[]{5, 3, 5, 6, 9, 13, 17},
+        null,
+        new int[]{1, 0, 1, 1, 1, 1, 1},
+        new int[]{2, 1, 1, 1, 1, 1, 1}
+    );
+
+    assertPartsOfSpeech(analyzerDecompound,"가락지나물은 한국, 중국, 일본",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP}
+    );
+
+    assertPartsOfSpeech(analyzerDecompoundKeep,"가락지나물은 한국, 중국, 일본",
+        new POS.Type[]{POS.Type.COMPOUND, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG, POS.Tag.J, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP}
+    );
+  }
+
+  public void testPartOfSpeechsWithInflects() throws IOException {
+    assertAnalyzesTo(analyzer, "감싸여",
+        new String[]{"감싸여"},
+        new int[]{0},
+        new int[]{3},
+        new int[]{1}
+    );
+
+    assertPartsOfSpeech(analyzer, "감싸여",
+        new POS.Type[]{POS.Type.INFLECT},
+        new POS.Tag[]{POS.Tag.VV},
+        new POS.Tag[]{POS.Tag.E}
+    );
+
+    assertAnalyzesTo(analyzerDecompound, "감싸여",
+        new String[]{"감싸이", "어"},
+        new int[]{0, 0},
+        new int[]{3, 3},
+        new int[]{1, 1}
+    );
+
+    assertAnalyzesTo(analyzerDecompoundKeep, "감싸여",
+        new String[]{"감싸여", "감싸이", "어"},
+        new int[]{0, 0, 0},
+        new int[]{3, 3, 3},
+        null,
+        new int[]{1, 0, 1},
+        new int[]{2, 1, 1}
+    );
+
+    assertPartsOfSpeech(analyzerDecompound, "감싸여",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.VV, POS.Tag.E},
+        new POS.Tag[]{POS.Tag.VV, POS.Tag.E}
+    );
+
+    assertPartsOfSpeech(analyzerDecompoundKeep, "감싸여",
+        new POS.Type[]{POS.Type.INFLECT, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.VV, POS.Tag.VV, POS.Tag.E},
+        new POS.Tag[]{POS.Tag.E, POS.Tag.VV, POS.Tag.E}
+    );
+  }
+
+  public void testUnknownWord() throws IOException {
+    assertAnalyzesTo(analyzer,"2018 평창 동계올림픽대회",
+        new String[]{"2018", "평창", "동계", "올림픽", "대회"},
+        new int[]{0, 5, 8, 10, 13},
+        new int[]{4, 7, 10, 13, 15},
+        new int[]{1, 1, 1, 1, 1});
+
+    assertPartsOfSpeech(analyzer,"2018 평창 동계올림픽대회",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.SN, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNG},
+        new POS.Tag[]{POS.Tag.SN, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNG}
+    );
+
+    assertAnalyzesTo(analyzerUnigram,"2018 평창 동계올림픽대회",
+        new String[]{"2", "0", "1", "8", "평창", "동계", "올림픽", "대회"},
+        new int[]{0, 1, 2, 3, 5, 8, 10, 13},
+        new int[]{1, 2, 3, 4, 7, 10, 13, 15},
+        new int[]{1, 1, 1, 1, 1, 1, 1, 1});
+
+    assertPartsOfSpeech(analyzerUnigram,"2018 평창 동계올림픽대회",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME,},
+        new POS.Tag[]{POS.Tag.SY, POS.Tag.SY, POS.Tag.SY, POS.Tag.SY, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNG},
+        new POS.Tag[]{POS.Tag.SY, POS.Tag.SY, POS.Tag.SY, POS.Tag.SY, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNP, POS.Tag.NNG}
+    );
+  }
+
+  public void testReading() throws IOException {
+    assertReadings(analyzer, "喜悲哀歡", "희비애환");
+    assertReadings(analyzer, "五朔居廬", "오삭거려");
+    assertReadings(analyzer, "가늘라", new String[] {null});
+    assertAnalyzesTo(analyzerReading,"喜悲哀歡",
+        new String[]{"희비애환"},
+        new int[]{0},
+        new int[]{4},
+        new int[]{1});
+    assertAnalyzesTo(analyzerReading,"五朔居廬",
+        new String[]{"오삭거려"},
+        new int[]{0},
+        new int[]{4},
+        new int[]{1});
+    assertAnalyzesTo(analyzerReading,"가늘라",
+        new String[]{"가늘라"},
+        new int[]{0},
+        new int[]{3},
+        new int[]{1});
+  }
+
+  public void testUserDict() throws IOException {
+    assertAnalyzesTo(analyzer, "c++ 프로그래밍 언어",
+        new String[]{"c++", "프로그래밍", "언어"},
+        new int[]{0, 4, 10},
+        new int[]{3, 9, 12},
+        new int[]{1, 1, 1}
+    );
+
+    assertPartsOfSpeech(analyzer, "c++ 프로그래밍 언어",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG}
+    );
+
+    assertAnalyzesTo(analyzerDecompound, "정부세종청사",
+        new String[]{"정부", "세종", "청사"},
+        new int[]{0, 2, 4},
+        new int[]{2, 4, 6},
+        new int[]{1, 1, 1}
+    );
+
+    assertPartsOfSpeech(analyzerDecompound, "정부세종청사",
+        new POS.Type[]{POS.Type.MORPHEME, POS.Type.MORPHEME, POS.Type.MORPHEME},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG},
+        new POS.Tag[]{POS.Tag.NNG, POS.Tag.NNG, POS.Tag.NNG}
+    );
+  }
+
+
+  /** blast some random strings through the tokenizer */
+  public void testRandomStrings() throws Exception {
+    checkRandomData(random(), analyzer, 500*RANDOM_MULTIPLIER);
+    checkRandomData(random(), analyzerUnigram, 500*RANDOM_MULTIPLIER);
+    checkRandomData(random(), analyzerDecompound, 500*RANDOM_MULTIPLIER);
+  }
+
+  /** blast some random large strings through the tokenizer */
+  public void testRandomHugeStrings() throws Exception {
+    Random random = random();
+    checkRandomData(random, analyzer, 20*RANDOM_MULTIPLIER, 8192);
+    checkRandomData(random, analyzerUnigram, 20*RANDOM_MULTIPLIER, 8192);
+    checkRandomData(random, analyzerDecompound, 20*RANDOM_MULTIPLIER, 8192);
+  }
+
+  public void testRandomHugeStringsMockGraphAfter() throws Exception {
+    // Randomly inject graph tokens after KoreanTokenizer:
+    Random random = random();
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer tokenizer = new KoreanTokenizer(newAttributeFactory(), null, DecompoundMode.MIXED, false);
+        TokenStream graph = new MockGraphTokenFilter(random(), tokenizer);
+        return new TokenStreamComponents(tokenizer, graph);
+      }
+    };
+    checkRandomData(random, analyzer, 20*RANDOM_MULTIPLIER, 8192);
+    analyzer.close();
+  }
+
+  private void assertReadings(Analyzer analyzer, String input, String... readings) throws IOException {
+    try (TokenStream ts = analyzer.tokenStream("ignored", input)) {
+      ReadingAttribute readingAtt = ts.addAttribute(ReadingAttribute.class);
+      ts.reset();
+      for(String reading : readings) {
+        assertTrue(ts.incrementToken());
+        assertEquals(reading, readingAtt.getReading());
+      }
+      assertFalse(ts.incrementToken());
+      ts.end();
+    }
+  }
+
+  private void assertPartsOfSpeech(Analyzer analyzer, String input, POS.Type[] posTypes, POS.Tag[] leftPosTags, POS.Tag[] rightPosTags) throws IOException {
+    assert posTypes.length == leftPosTags.length && posTypes.length == rightPosTags.length;
+    try (TokenStream ts = analyzer.tokenStream("ignored", input)) {
+      PartOfSpeechAttribute partOfSpeechAtt = ts.addAttribute(PartOfSpeechAttribute.class);
+      ts.reset();
+      for (int i = 0; i < posTypes.length; i++) {
+        POS.Type posType = posTypes[i];
+        POS.Tag leftTag = leftPosTags[i];
+        POS.Tag rightTag = rightPosTags[i];
+        assertTrue(ts.incrementToken());
+        assertEquals(posType, partOfSpeechAtt.getPOSType());
+        assertEquals(leftTag, partOfSpeechAtt.getLeftPOS());
+        assertEquals(rightTag, partOfSpeechAtt.getRightPOS());
+      }
+      assertFalse(ts.incrementToken());
+      ts.end();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizerFactory.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizerFactory.java
new file mode 100644
index 0000000..755af64
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/TestKoreanTokenizerFactory.java
@@ -0,0 +1,113 @@
+/*
+ * 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.ko;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+
+/**
+ * Simple tests for {@link KoreanTokenizerFactory}
+ */
+public class TestKoreanTokenizerFactory extends BaseTokenStreamTestCase {
+  public void testSimple() throws IOException {
+    KoreanTokenizerFactory factory = new KoreanTokenizerFactory(Collections.emptyMap());
+    factory.inform(new StringMockResourceLoader(""));
+    TokenStream ts = factory.create(newAttributeFactory());
+    ((Tokenizer)ts).setReader(new StringReader("안녕하세요"));
+    assertTokenStreamContents(ts,
+        new String[] { "안녕", "하", "시", "어요" },
+        new int[] { 0, 2, 3, 3 },
+        new int[] { 2, 3, 5, 5 }
+    );
+  }
+
+  /**
+   * Test decompoundMode
+   */
+  public void testDiscardDecompound() throws IOException {
+    Map<String,String> args = new HashMap<>();
+    args.put("decompoundMode", "discard");
+    KoreanTokenizerFactory factory = new KoreanTokenizerFactory(args);
+    factory.inform(new StringMockResourceLoader(""));
+    TokenStream ts = factory.create(newAttributeFactory());
+    ((Tokenizer)ts).setReader(new StringReader("갠지스강"));
+    assertTokenStreamContents(ts,
+        new String[] { "갠지스", "강" }
+    );
+  }
+
+  public void testNoDecompound() throws IOException {
+    Map<String,String> args = new HashMap<>();
+    args.put("decompoundMode", "none");
+    KoreanTokenizerFactory factory = new KoreanTokenizerFactory(args);
+    factory.inform(new StringMockResourceLoader(""));
+    TokenStream ts = factory.create(newAttributeFactory());
+    ((Tokenizer)ts).setReader(new StringReader("갠지스강"));
+    assertTokenStreamContents(ts,
+        new String[] { "갠지스강" }
+    );
+  }
+
+  public void testMixedDecompound() throws IOException {
+    Map<String,String> args = new HashMap<>();
+    args.put("decompoundMode", "mixed");
+    KoreanTokenizerFactory factory = new KoreanTokenizerFactory(args);
+    factory.inform(new StringMockResourceLoader(""));
+    TokenStream ts = factory.create(newAttributeFactory());
+    ((Tokenizer)ts).setReader(new StringReader("갠지스강"));
+    assertTokenStreamContents(ts,
+        new String[] { "갠지스강", "갠지스", "강" }
+    );
+  }
+
+  /**
+   * Test user dictionary
+   */
+  public void testUserDict() throws IOException {
+    String userDict =
+        "# Additional nouns\n" +
+        "세종시 세종 시\n" +
+        "# \n" +
+        "c++\n";
+    Map<String,String> args = new HashMap<>();
+    args.put("userDictionary", "userdict.txt");
+    KoreanTokenizerFactory factory = new KoreanTokenizerFactory(args);
+    factory.inform(new StringMockResourceLoader(userDict));
+    TokenStream ts = factory.create(newAttributeFactory());
+    ((Tokenizer)ts).setReader(new StringReader("세종시"));
+    assertTokenStreamContents(ts,
+        new String[] { "세종", "시" }
+    );
+  }
+
+  /** Test that bogus arguments result in exception */
+  public void testBogusArguments() throws Exception {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      new KoreanTokenizerFactory(new HashMap<String,String>() {{
+        put("bogusArg", "bogusValue");
+      }});
+    });
+    assertTrue(expected.getMessage().contains("Unknown parameters"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/TestTokenInfoDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/TestTokenInfoDictionary.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/TestTokenInfoDictionary.java
new file mode 100644
index 0000000..d278841
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/TestTokenInfoDictionary.java
@@ -0,0 +1,113 @@
+/*
+ * 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.ko.dict;
+
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.IntsRefFSTEnum;
+import org.apache.lucene.util.fst.IntsRefFSTEnum.InputOutput;
+
+public class TestTokenInfoDictionary extends LuceneTestCase {
+
+  /** enumerates the entire FST/lookup data and just does basic sanity checks */
+  public void testEnumerateAll() throws Exception {
+    // just for debugging
+    int numTerms = 0;
+    int numWords = 0;
+    int lastWordId = -1;
+    int lastSourceId = -1;
+    CharacterDefinition charDef = CharacterDefinition.getInstance();
+    TokenInfoDictionary tid = TokenInfoDictionary.getInstance();
+    ConnectionCosts matrix = ConnectionCosts.getInstance();
+    FST<Long> fst = tid.getFST().getInternalFST();
+    IntsRefFSTEnum<Long> fstEnum = new IntsRefFSTEnum<>(fst);
+    InputOutput<Long> mapping;
+    IntsRef scratch = new IntsRef();
+    while ((mapping = fstEnum.next()) != null) {
+      numTerms++;
+      IntsRef input = mapping.input;
+      char chars[] = new char[input.length];
+      for (int i = 0; i < chars.length; i++) {
+        chars[i] = (char)input.ints[input.offset+i];
+      }
+      String surfaceForm = new String(chars);
+      assertTrue(UnicodeUtil.validUTF16String(surfaceForm));
+      
+      Long output = mapping.output;
+      int sourceId = output.intValue();
+      // we walk in order, terms, sourceIds, and wordIds should always be increasing
+      assertTrue(sourceId > lastSourceId);
+      lastSourceId = sourceId;
+      tid.lookupWordIds(sourceId, scratch);
+      for (int i = 0; i < scratch.length; i++) {
+        numWords++;
+        int wordId = scratch.ints[scratch.offset+i];
+        assertTrue(wordId > lastWordId);
+        lastWordId = wordId;
+
+        int leftId = tid.getLeftId(wordId);
+        int rightId = tid.getRightId(wordId);
+
+        matrix.get(rightId, leftId);
+
+        tid.getWordCost(wordId);
+
+        POS.Type type = tid.getPOSType(wordId);
+        POS.Tag leftPOS = tid.getLeftPOS(wordId);
+        POS.Tag rightPOS = tid.getRightPOS(wordId);
+
+        if (type == POS.Type.MORPHEME) {
+          assertTrue(leftPOS == rightPOS);
+          String reading = tid.getReading(wordId);
+          boolean isHanja = charDef.isHanja(surfaceForm.charAt(0));
+          if (isHanja) {
+            assertTrue(reading != null);
+            for (int j = 0; j < reading.length(); j++) {
+              assertTrue(charDef.isHangul(reading.charAt(j)));
+            }
+          }
+          if (reading != null) {
+            assertTrue(UnicodeUtil.validUTF16String(reading));
+          }
+        } else {
+          if (type == POS.Type.COMPOUND) {
+            assertTrue(leftPOS == rightPOS);
+            assertTrue(leftPOS == POS.Tag.NNG || rightPOS == POS.Tag.NNP);
+          }
+          Dictionary.Morpheme[] decompound = tid.getMorphemes(wordId,  chars, 0, chars.length);
+          if (decompound != null) {
+            int offset = 0;
+            for (Dictionary.Morpheme morph : decompound) {
+              assertTrue(UnicodeUtil.validUTF16String(morph.surfaceForm));
+              if (type != POS.Type.INFLECT) {
+                assertEquals(morph.surfaceForm, surfaceForm.substring(offset, offset + morph.surfaceForm.length()));
+                offset += morph.surfaceForm.length();
+              }
+            }
+            assertTrue(offset <= surfaceForm.length());
+          }
+        }
+      }
+    }
+    if (VERBOSE) {
+      System.out.println("checked " + numTerms + " terms, " + numWords + " words.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/UserDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/UserDictionaryTest.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/UserDictionaryTest.java
new file mode 100644
index 0000000..b008cf3
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/dict/UserDictionaryTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.ko.dict;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.analysis.ko.TestKoreanTokenizer;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+public class UserDictionaryTest extends LuceneTestCase {
+  @Test
+  public void testLookup() throws IOException {
+    UserDictionary dictionary = TestKoreanTokenizer.readDict();
+    String s = "세종";
+    char[] sArray = s.toCharArray();
+    List<Integer> wordIds = dictionary.lookup(sArray, 0, s.length());
+    assertEquals(1, wordIds.size());
+    assertNull(dictionary.getMorphemes(wordIds.get(0), sArray, 0, s.length()));
+
+    s = "세종시";
+    sArray = s.toCharArray();
+    wordIds = dictionary.lookup(sArray, 0, s.length());
+    assertEquals(2, wordIds.size());
+    assertNull(dictionary.getMorphemes(wordIds.get(0), sArray, 0, s.length()));
+
+    Dictionary.Morpheme[] decompound = dictionary.getMorphemes(wordIds.get(1), sArray, 0, s.length());
+    assertTrue(decompound.length == 2);
+    assertEquals(decompound[0].posTag, POS.Tag.NNG);
+    assertEquals(decompound[0].surfaceForm, "세종");
+    assertEquals(decompound[1].posTag, POS.Tag.NNG);
+    assertEquals(decompound[1].surfaceForm, "시");
+
+    s = "c++";
+    sArray = s.toCharArray();
+    wordIds = dictionary.lookup(sArray, 0, s.length());
+    assertEquals(1, wordIds.size());
+    assertNull(dictionary.getMorphemes(wordIds.get(0), sArray, 0, s.length()));
+  }
+  
+  @Test
+  public void testRead() throws IOException {
+    UserDictionary dictionary = TestKoreanTokenizer.readDict();
+    assertNotNull(dictionary);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/userdict.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/userdict.txt b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/userdict.txt
new file mode 100644
index 0000000..63c1c3a
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/userdict.txt
@@ -0,0 +1,5 @@
+# Additional nouns
+c++
+C샤프
+세종
+세종시 세종 시
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/BinaryDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/BinaryDictionaryWriter.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/BinaryDictionaryWriter.java
new file mode 100644
index 0000000..35c16ae
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/BinaryDictionaryWriter.java
@@ -0,0 +1,282 @@
+/*
+ * 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.ko.util;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.analysis.ko.dict.Dictionary;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.ArrayUtil;
+
+import org.apache.lucene.analysis.ko.dict.BinaryDictionary;
+
+public abstract class BinaryDictionaryWriter {
+  protected final Class<? extends BinaryDictionary> implClazz;
+  protected ByteBuffer buffer;
+  private int targetMapEndOffset = 0, lastWordId = -1, lastSourceId = -1;
+  private int[] targetMap = new int[8192];
+  private int[] targetMapOffsets = new int[8192];
+  private final ArrayList<String> posDict = new ArrayList<>();
+
+  public BinaryDictionaryWriter(Class<? extends BinaryDictionary> implClazz, int size) {
+    this.implClazz = implClazz;
+    buffer = ByteBuffer.allocate(size);
+  }
+  
+  /**
+   * put the entry in map
+   *
+   * mecab-ko-dic features
+   *
+   * 0   - surface
+   * 1   - left cost
+   * 2   - right cost
+   * 3   - word cost
+   * 4   - part of speech0+part of speech1+...
+   * 5   - semantic class
+   * 6   - T if the last character of the surface form has a coda, F otherwise
+   * 7   - reading
+   * 8   - POS type (*, Compound, Inflect, Preanalysis)
+   * 9   - left POS
+   * 10  - right POS
+   * 11  - expression
+   *
+   * @return current position of buffer, which will be wordId of next entry
+   */
+  public int put(String[] entry) {
+    short leftId = Short.parseShort(entry[1]);
+    short rightId = Short.parseShort(entry[2]);
+    short wordCost = Short.parseShort(entry[3]);
+
+    final POS.Type posType = POS.resolveType(entry[8]);
+    final POS.Tag leftPOS;
+    final POS.Tag rightPOS;
+    if (posType == POS.Type.MORPHEME || posType == POS.Type.COMPOUND || entry[9].equals("*")) {
+      leftPOS = POS.resolveTag(entry[4]);
+      assert (entry[9].equals("*") && entry[10].equals("*"));
+      rightPOS = leftPOS;
+    } else {
+      leftPOS = POS.resolveTag(entry[9]);
+      rightPOS = POS.resolveTag(entry[10]);
+    }
+    final String reading = entry[7].equals("*") ? "" : entry[0].equals(entry[7]) ? "" : entry[7];
+    final String expression = entry[11].equals("*") ? "" : entry[11];
+
+    // extend buffer if necessary
+    int left = buffer.remaining();
+    // worst case, 3 short + 4 bytes and features (all as utf-16)
+    int worstCase = 9 + 2*(expression.length() + reading.length());
+    if (worstCase > left) {
+      ByteBuffer newBuffer = ByteBuffer.allocate(ArrayUtil.oversize(buffer.limit() + worstCase - left, 1));
+      buffer.flip();
+      newBuffer.put(buffer);
+      buffer = newBuffer;
+    }
+
+    // add pos mapping
+    int toFill = 1+leftId - posDict.size();
+    for (int i = 0; i < toFill; i++) {
+      posDict.add(null);
+    }
+    String fullPOSData = leftPOS.name() + "," + entry[5];
+    String existing = posDict.get(leftId);
+    assert existing == null || existing.equals(fullPOSData);
+    posDict.set(leftId, fullPOSData);
+
+    final Dictionary.Morpheme[] morphemes;
+    // true if the POS and decompounds of the token are all the same.
+    boolean hasSinglePOS = (leftPOS == rightPOS);
+    if (posType != POS.Type.MORPHEME && expression.length() > 0) {
+      String[] exprTokens = expression.split("\\+");
+      morphemes = new Dictionary.Morpheme[exprTokens.length];
+      for (int i = 0; i < exprTokens.length; i++) {
+        String[] tokenSplit = exprTokens[i].split("\\/");
+        assert tokenSplit.length == 3;
+        POS.Tag exprTag = POS.resolveTag(tokenSplit[1]);
+        morphemes[i] = new Dictionary.Morpheme(exprTag, tokenSplit[0]);
+        if (leftPOS != exprTag) {
+          hasSinglePOS = false;
+        }
+      }
+    } else {
+      morphemes = new Dictionary.Morpheme[0];
+    }
+
+    int flags = 0;
+    if (hasSinglePOS) {
+      flags |= BinaryDictionary.HAS_SINGLE_POS;
+    }
+    if (posType == POS.Type.MORPHEME && reading.length() > 0) {
+      flags |= BinaryDictionary.HAS_READING;
+    }
+
+    assert leftId < 8192; // there are still unused bits
+    assert posType.ordinal() < 4;
+    buffer.putShort((short)(leftId << 2 | posType.ordinal()));
+    buffer.putShort((short) (rightId << 2 | flags));
+    buffer.putShort(wordCost);
+
+    if (posType == POS.Type.MORPHEME) {
+      assert leftPOS == rightPOS;
+      if (reading.length() > 0) {
+        writeString(reading);
+      }
+    } else {
+      if (hasSinglePOS == false) {
+        buffer.put((byte) rightPOS.ordinal());
+      }
+      buffer.put((byte) morphemes.length);
+      int compoundOffset = 0;
+      for (int i = 0; i < morphemes.length; i++) {
+        if (hasSinglePOS == false) {
+          buffer.put((byte) morphemes[i].posTag.ordinal());
+        }
+        if (posType != POS.Type.INFLECT) {
+          buffer.put((byte) morphemes[i].surfaceForm.length());
+          compoundOffset += morphemes[i].surfaceForm.length();
+        } else {
+          writeString(morphemes[i].surfaceForm);
+        }
+        assert compoundOffset <= entry[0].length() : Arrays.toString(entry);
+      }
+    }
+    return buffer.position();
+  }
+
+  private void writeString(String s) {
+    buffer.put((byte) s.length());
+    for (int i = 0; i < s.length(); i++) {
+      buffer.putChar(s.charAt(i));
+    }
+  }
+
+  public void addMapping(int sourceId, int wordId) {
+    assert wordId > lastWordId : "words out of order: " + wordId + " vs lastID: " + lastWordId;
+
+    if (sourceId > lastSourceId) {
+      assert sourceId > lastSourceId : "source ids out of order: lastSourceId=" + lastSourceId + " vs sourceId=" + sourceId;
+      targetMapOffsets = ArrayUtil.grow(targetMapOffsets, sourceId + 1);
+      for (int i = lastSourceId + 1; i <= sourceId; i++) {
+        targetMapOffsets[i] = targetMapEndOffset;
+      }
+    } else {
+      assert sourceId == lastSourceId;
+    }
+
+    targetMap = ArrayUtil.grow(targetMap, targetMapEndOffset + 1);
+    targetMap[targetMapEndOffset] = wordId;
+    targetMapEndOffset++;
+
+    lastSourceId = sourceId;
+    lastWordId = wordId;
+  }
+  
+  protected final String getBaseFileName(String baseDir) {
+    return baseDir + File.separator + implClazz.getName().replace('.', File.separatorChar);
+  }
+
+  /**
+   * Write dictionary in file
+   * @throws IOException if an I/O error occurs writing the dictionary files
+   */
+  public void write(String baseDir) throws IOException {
+    final String baseName = getBaseFileName(baseDir);
+    writeDictionary(baseName + BinaryDictionary.DICT_FILENAME_SUFFIX);
+    writeTargetMap(baseName + BinaryDictionary.TARGETMAP_FILENAME_SUFFIX);
+    writePosDict(baseName + BinaryDictionary.POSDICT_FILENAME_SUFFIX);
+  }
+
+  protected void writeTargetMap(String filename) throws IOException {
+    new File(filename).getParentFile().mkdirs();
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, BinaryDictionary.TARGETMAP_HEADER, BinaryDictionary.VERSION);
+
+      final int numSourceIds = lastSourceId + 1;
+      out.writeVInt(targetMapEndOffset); // <-- size of main array
+      out.writeVInt(numSourceIds + 1); // <-- size of offset array (+ 1 more entry)
+      int prev = 0, sourceId = 0;
+      for (int ofs = 0; ofs < targetMapEndOffset; ofs++) {
+        final int val = targetMap[ofs], delta = val - prev;
+        assert delta >= 0;
+        if (ofs == targetMapOffsets[sourceId]) {
+          out.writeVInt((delta << 1) | 0x01);
+          sourceId++;
+        } else {
+          out.writeVInt((delta << 1));
+        }
+        prev += delta;
+      }
+      assert sourceId == numSourceIds : "sourceId:"+sourceId+" != numSourceIds:"+numSourceIds;
+    } finally {
+      os.close();
+    }
+  }
+
+  protected void writePosDict(String filename) throws IOException {
+    new File(filename).getParentFile().mkdirs();
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, BinaryDictionary.POSDICT_HEADER, BinaryDictionary.VERSION);
+      out.writeVInt(posDict.size());
+      for (String s : posDict) {
+        if (s == null) {
+          out.writeByte((byte) POS.Tag.UNKNOWN.ordinal());
+        } else {
+          String data[] = CSVUtil.parse(s);
+          assert data.length == 2 : "malformed pos/semanticClass: " + s;
+          out.writeByte((byte) POS.Tag.valueOf(data[0]).ordinal());
+        }
+      }
+    } finally {
+      os.close();
+    }
+  }
+  
+  protected void writeDictionary(String filename) throws IOException {
+    new File(filename).getParentFile().mkdirs();
+    final FileOutputStream os = new FileOutputStream(filename);
+    try {
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, BinaryDictionary.DICT_HEADER, BinaryDictionary.VERSION);
+      out.writeVInt(buffer.position());
+      final WritableByteChannel channel = Channels.newChannel(os);
+      // Write Buffer
+      buffer.flip();  // set position to 0, set limit to current position
+      channel.write(buffer);
+      assert buffer.remaining() == 0L;
+    } finally {
+      os.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/CharacterDefinitionWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/CharacterDefinitionWriter.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/CharacterDefinitionWriter.java
new file mode 100644
index 0000000..5a78549
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/CharacterDefinitionWriter.java
@@ -0,0 +1,94 @@
+/*
+ * 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.ko.util;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+public final class CharacterDefinitionWriter {
+
+  private final byte[] characterCategoryMap = new byte[0x10000];
+  
+  private final boolean[] invokeMap = new boolean[CharacterDefinition.CLASS_COUNT];
+  private final boolean[] groupMap = new boolean[CharacterDefinition.CLASS_COUNT];
+    
+  /**
+   * Constructor for building. TODO: remove write access
+   */
+  public CharacterDefinitionWriter() {
+    Arrays.fill(characterCategoryMap, CharacterDefinition.DEFAULT);
+  }
+  
+  /**
+   * Put mapping from unicode code point to character class.
+   * 
+   * @param codePoint
+   *            code point
+   * @param characterClassName character class name
+   */
+  public void putCharacterCategory(int codePoint, String characterClassName) {
+    characterClassName = characterClassName.split(" ")[0]; // use first
+    // category
+    // class
+    
+    // Override Nakaguro
+    if (codePoint == 0x30FB) {
+      characterClassName = "SYMBOL";
+    }
+    characterCategoryMap[codePoint] = CharacterDefinition.lookupCharacterClass(characterClassName);
+  }
+  
+  public void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
+    final byte characterClass = CharacterDefinition.lookupCharacterClass(characterClassName);
+    invokeMap[characterClass] = invoke == 1;
+    groupMap[characterClass] = group == 1;
+    // TODO: length def ignored
+  }
+  
+  public void write(String baseDir) throws IOException {
+    String filename = baseDir + File.separator +
+      CharacterDefinition.class.getName().replace('.', File.separatorChar) + CharacterDefinition.FILENAME_SUFFIX;
+    new File(filename).getParentFile().mkdirs();
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, CharacterDefinition.HEADER, CharacterDefinition.VERSION);
+      out.writeBytes(characterCategoryMap, 0, characterCategoryMap.length);
+      for (int i = 0; i < CharacterDefinition.CLASS_COUNT; i++) {
+        final byte b = (byte) (
+          (invokeMap[i] ? 0x01 : 0x00) | 
+          (groupMap[i] ? 0x02 : 0x00)
+        );
+        out.writeByte(b);
+      }
+    } finally {
+      os.close();
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsBuilder.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsBuilder.java
new file mode 100644
index 0000000..29659de
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsBuilder.java
@@ -0,0 +1,67 @@
+/*
+ * 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.ko.util;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
+
+public class ConnectionCostsBuilder {
+  
+  private ConnectionCostsBuilder() {
+  }
+  
+  public static ConnectionCostsWriter build(String filename) throws IOException {
+    FileInputStream inputStream = new FileInputStream(filename);
+    Charset cs = StandardCharsets.US_ASCII;
+    CharsetDecoder decoder = cs.newDecoder()
+        .onMalformedInput(CodingErrorAction.REPORT)
+        .onUnmappableCharacter(CodingErrorAction.REPORT);
+    InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
+    LineNumberReader lineReader = new LineNumberReader(streamReader);
+    
+    String line = lineReader.readLine();
+    String[] dimensions = line.split("\\s+");
+    
+    assert dimensions.length == 2;
+    
+    int forwardSize = Integer.parseInt(dimensions[0]);
+    int backwardSize = Integer.parseInt(dimensions[1]);
+    
+    assert forwardSize > 0 && backwardSize > 0;
+    
+    ConnectionCostsWriter costs = new ConnectionCostsWriter(forwardSize, backwardSize);
+    
+    while ((line = lineReader.readLine()) != null) {
+      String[] fields = line.split("\\s+");
+      
+      assert fields.length == 3;
+      
+      int forwardId = Integer.parseInt(fields[0]);
+      int backwardId = Integer.parseInt(fields[1]);
+      int cost = Integer.parseInt(fields[2]);
+      
+      costs.add(forwardId, backwardId, cost);
+    }
+    return costs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsWriter.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsWriter.java
new file mode 100644
index 0000000..f16f827
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/ConnectionCostsWriter.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ko.util;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.lucene.analysis.ko.dict.ConnectionCosts;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+public final class ConnectionCostsWriter {
+  
+  private final short[][] costs; // array is backward IDs first since get is called using the same backward ID consecutively. maybe doesn't matter.
+  private final int forwardSize;
+  private final int backwardSize;
+  /**
+   * Constructor for building. TODO: remove write access
+   */
+  public ConnectionCostsWriter(int forwardSize, int backwardSize) {
+    this.forwardSize = forwardSize;
+    this.backwardSize = backwardSize;
+    this.costs = new short[backwardSize][forwardSize];
+  }
+  
+  public void add(int forwardId, int backwardId, int cost) {
+    this.costs[backwardId][forwardId] = (short)cost;
+  }
+  
+  public void write(String baseDir) throws IOException {
+    String filename = baseDir + File.separator +
+      ConnectionCosts.class.getName().replace('.', File.separatorChar) + ConnectionCosts.FILENAME_SUFFIX;
+    new File(filename).getParentFile().mkdirs();
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, ConnectionCosts.HEADER, ConnectionCosts.VERSION);
+      out.writeVInt(forwardSize);
+      out.writeVInt(backwardSize);
+      int last = 0;
+      assert costs.length == backwardSize;
+      for (short[] a : costs) {
+        assert a.length == forwardSize;
+        for (int i = 0; i < a.length; i++) {
+          int delta = (int)a[i] - last;
+          out.writeZInt(delta);
+          last = a[i];
+        }
+      }
+    } finally {
+      os.close();
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/DictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/DictionaryBuilder.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/DictionaryBuilder.java
new file mode 100644
index 0000000..e0039a2
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/DictionaryBuilder.java
@@ -0,0 +1,67 @@
+/*
+ * 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.ko.util;
+
+import java.io.File;
+import java.io.IOException;
+
+public class DictionaryBuilder {
+
+  private DictionaryBuilder() {
+  }
+  
+  public static void build(String inputDirname, String outputDirname, String encoding, boolean normalizeEntry) throws IOException {
+    System.out.println("building tokeninfo dict...");
+    TokenInfoDictionaryBuilder tokenInfoBuilder = new TokenInfoDictionaryBuilder(encoding, normalizeEntry);
+    TokenInfoDictionaryWriter tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
+    tokenInfoDictionary.write(outputDirname);
+    tokenInfoDictionary = null;
+    tokenInfoBuilder = null;
+    System.out.println("done");
+    
+    System.out.print("building unknown word dict...");
+    UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
+    UnknownDictionaryWriter unkDictionary = unkBuilder.build(inputDirname);
+    unkDictionary.write(outputDirname);
+    unkDictionary = null;
+    unkBuilder = null;
+    System.out.println("done");
+    
+    System.out.print("building connection costs...");
+    ConnectionCostsWriter connectionCosts
+      = ConnectionCostsBuilder.build(inputDirname + File.separator + "matrix.def");
+    connectionCosts.write(outputDirname);
+    System.out.println("done");
+  }
+  
+  public static void main(String[] args) throws IOException {
+    String inputDirname = args[0];
+    String outputDirname = args[1];
+    String inputEncoding = args[2];
+    boolean normalizeEntries = Boolean.parseBoolean(args[3]);
+    
+    System.out.println("dictionary builder");
+    System.out.println("");
+    System.out.println("input directory: " + inputDirname);
+    System.out.println("output directory: " + outputDirname);
+    System.out.println("input encoding: " + inputEncoding);
+    System.out.println("normalize entries: " + normalizeEntries);
+    System.out.println("");
+    DictionaryBuilder.build(inputDirname, outputDirname, inputEncoding, normalizeEntries);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryBuilder.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryBuilder.java
new file mode 100644
index 0000000..de60daa
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryBuilder.java
@@ -0,0 +1,150 @@
+/*
+ * 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.ko.util;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+
+import com.ibm.icu.text.Normalizer2;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+
+public class TokenInfoDictionaryBuilder {
+  
+  /** Internal word id - incrementally assigned as entries are read and added. This will be byte offset of dictionary file */
+  private int offset = 0;
+  
+  private String encoding = "utf-8";
+  
+  private boolean normalizeEntries = false;
+  private Normalizer2 normalizer;
+
+  public TokenInfoDictionaryBuilder(String encoding, boolean normalizeEntries) {
+    this.encoding = encoding;
+    this.normalizeEntries = normalizeEntries;
+    this.normalizer = normalizeEntries ? Normalizer2.getInstance(null, "nfkc", Normalizer2.Mode.COMPOSE) : null;
+  }
+  
+  public TokenInfoDictionaryWriter build(String dirname) throws IOException {
+    FilenameFilter filter = (dir, name) -> name.endsWith(".csv");
+    ArrayList<File> csvFiles = new ArrayList<>();
+    for (File file : new File(dirname).listFiles(filter)) {
+      csvFiles.add(file);
+    }
+    Collections.sort(csvFiles);
+    return buildDictionary(csvFiles);
+  }
+
+  public TokenInfoDictionaryWriter buildDictionary(List<File> csvFiles) throws IOException {
+    TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
+    
+    // all lines in the file
+    System.out.println("  parse...");
+    List<String[]> lines = new ArrayList<>(400000);
+    for (File file : csvFiles){
+      FileInputStream inputStream = new FileInputStream(file);
+      Charset cs = Charset.forName(encoding);
+      CharsetDecoder decoder = cs.newDecoder()
+          .onMalformedInput(CodingErrorAction.REPORT)
+          .onUnmappableCharacter(CodingErrorAction.REPORT);
+      InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
+      BufferedReader reader = new BufferedReader(streamReader);
+      
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        String[] entry = CSVUtil.parse(line);
+
+        if(entry.length < 12) {
+          System.out.println("Entry in CSV is not valid: " + line);
+          continue;
+        }
+
+        // NFKC normalize dictionary entry
+        if (normalizeEntries) {
+          String[] normalizedEntry = new String[entry.length];
+          for (int i = 0; i < entry.length; i++) {
+            normalizedEntry[i] = normalizer.normalize(entry[i]);
+          }
+          lines.add(normalizedEntry);
+        } else {
+          lines.add(entry);
+        }
+      }
+    }
+    
+    System.out.println("  sort...");
+
+    // sort by term: we sorted the files already and use a stable sort.
+    Collections.sort(lines, Comparator.comparing(left -> left[0]));
+    
+    System.out.println("  encode...");
+
+    PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton();
+    Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, true, 15);
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    long ord = -1; // first ord will be 0
+    String lastValue = null;
+
+    // build tokeninfo dictionary
+    for (String[] entry : lines) {
+      int next = dictionary.put(entry);
+
+      if(next == offset){
+        System.out.println("Failed to process line: " + Arrays.toString(entry));
+        continue;
+      }
+
+      String token = entry[0];
+      if (!token.equals(lastValue)) {
+        // new word to add to fst
+        ord++;
+        lastValue = token;
+        scratch.grow(token.length());
+        scratch.setLength(token.length());
+        for (int i = 0; i < token.length(); i++) {
+          scratch.setIntAt(i, (int) token.charAt(i));
+        }
+        fstBuilder.add(scratch.get(), ord);
+      }
+      dictionary.addMapping((int)ord, offset);
+      offset = next;
+    }
+
+    final FST<Long> fst = fstBuilder.finish();
+    
+    System.out.print("  " + fstBuilder.getNodeCount() + " nodes, " + fstBuilder.getArcCount() + " arcs, " + fst.ramBytesUsed() + " bytes...  ");
+    dictionary.setFST(fst);
+    System.out.println(" done");
+    
+    return dictionary;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryWriter.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryWriter.java
new file mode 100644
index 0000000..c1554d2
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/TokenInfoDictionaryWriter.java
@@ -0,0 +1,49 @@
+/*
+ * 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.ko.util;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.apache.lucene.analysis.ko.dict.TokenInfoDictionary;
+import org.apache.lucene.util.fst.FST;
+
+public class TokenInfoDictionaryWriter extends BinaryDictionaryWriter {
+  private FST<Long> fst;
+
+  public TokenInfoDictionaryWriter(int size) {
+    super(TokenInfoDictionary.class, size);
+  }
+  
+  public void setFST(FST<Long> fst) {
+    this.fst = fst;
+  }
+  
+  @Override
+  public void write(String baseDir) throws IOException {
+    super.write(baseDir);
+    writeFST(getBaseFileName(baseDir) + TokenInfoDictionary.FST_FILENAME_SUFFIX);
+  }
+  
+  protected void writeFST(String filename) throws IOException {
+    Path p = Paths.get(filename);
+    Files.createDirectories(p.getParent());
+    fst.save(p);
+  }  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryBuilder.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryBuilder.java
new file mode 100644
index 0000000..a408866
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryBuilder.java
@@ -0,0 +1,134 @@
+/*
+ * 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.ko.util;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
+
+public class UnknownDictionaryBuilder {
+  private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,1798,3559,3677,SY,*,*,*,*,*,*,*";
+  
+  private String encoding = "utf-8";
+  
+  public UnknownDictionaryBuilder(String encoding) {
+    this.encoding = encoding;
+  }
+  
+  public UnknownDictionaryWriter build(String dirname) throws IOException {
+    UnknownDictionaryWriter unkDictionary = readDictionaryFile(dirname + File.separator + "unk.def");  //Should be only one file
+    readCharacterDefinition(dirname + File.separator + "char.def", unkDictionary);
+    return unkDictionary;
+  }
+  
+  public UnknownDictionaryWriter readDictionaryFile(String filename)
+      throws IOException {
+    return readDictionaryFile(filename, encoding);
+  }
+  
+  public UnknownDictionaryWriter readDictionaryFile(String filename, String encoding)
+      throws IOException {
+    UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
+    
+    FileInputStream inputStream = new FileInputStream(filename);
+    Charset cs = Charset.forName(encoding);
+    CharsetDecoder decoder = cs.newDecoder()
+        .onMalformedInput(CodingErrorAction.REPORT)
+        .onUnmappableCharacter(CodingErrorAction.REPORT);
+    InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
+    LineNumberReader lineReader = new LineNumberReader(streamReader);
+    
+    dictionary.put(CSVUtil.parse(NGRAM_DICTIONARY_ENTRY));
+    
+    List<String[]> lines = new ArrayList<>();
+    String line = null;
+    while ((line = lineReader.readLine()) != null) {
+      // note: unk.def only has 10 fields, it simplifies the writer to just append empty reading and pronunciation,
+      // even though the unknown dictionary returns hardcoded null here.
+      final String[] parsed = CSVUtil.parse(line + ",*,*"); // Probably we don't need to validate entry
+      lines.add(parsed);
+    }
+    
+    Collections.sort(lines, new Comparator<String[]>() {
+      public int compare(String[] left, String[] right) {
+        int leftId = CharacterDefinition.lookupCharacterClass(left[0]);
+        int rightId = CharacterDefinition.lookupCharacterClass(right[0]);
+        return leftId - rightId;
+      }
+    });
+    
+    for (String[] entry : lines) {
+      dictionary.put(entry);
+    }
+    
+    return dictionary;
+  }
+  
+  public void readCharacterDefinition(String filename, UnknownDictionaryWriter dictionary) throws IOException {
+    FileInputStream inputStream = new FileInputStream(filename);
+    InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
+    LineNumberReader lineReader = new LineNumberReader(streamReader);
+    
+    String line = null;
+    
+    while ((line = lineReader.readLine()) != null) {
+      line = line.replaceAll("^\\s", "");
+      line = line.replaceAll("\\s*#.*", "");
+      line = line.replaceAll("\\s+", " ");
+      
+      // Skip empty line or comment line
+      if(line.length() == 0) {
+        continue;
+      }
+      
+      if(line.startsWith("0x")) {  // Category mapping
+        String[] values = line.split(" ", 2);  // Split only first space
+        
+        if(!values[0].contains("..")) {
+          int cp = Integer.decode(values[0]).intValue();
+          dictionary.putCharacterCategory(cp, values[1]);
+        } else {
+          String[] codePoints = values[0].split("\\.\\.");
+          int cpFrom = Integer.decode(codePoints[0]).intValue();
+          int cpTo = Integer.decode(codePoints[1]).intValue();
+          
+          for(int i = cpFrom; i <= cpTo; i++){
+            dictionary.putCharacterCategory(i, values[1]);
+          }
+        }
+      } else {  // Invoke definition
+        String[] values = line.split(" "); // Consecutive space is merged above
+        String characterClassName = values[0];
+        int invoke = Integer.parseInt(values[1]);
+        int group = Integer.parseInt(values[2]);
+        int length = Integer.parseInt(values[3]);
+        dictionary.putInvokeDefinition(characterClassName, invoke, group, length);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryWriter.java b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryWriter.java
new file mode 100644
index 0000000..ff98a8d
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/java/org/apache/lucene/analysis/ko/util/UnknownDictionaryWriter.java
@@ -0,0 +1,65 @@
+/*
+ * 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.ko.util;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
+import org.apache.lucene.analysis.ko.dict.UnknownDictionary;
+
+public class UnknownDictionaryWriter extends BinaryDictionaryWriter {
+
+  private final CharacterDefinitionWriter characterDefinition = new CharacterDefinitionWriter();
+
+  public UnknownDictionaryWriter(int size) {
+    super(UnknownDictionary.class, size);
+  }
+  
+  @Override
+  public int put(String[] entry) {
+    // Get wordId of current entry
+    int wordId = buffer.position();
+    
+    // Put entry
+    int result = super.put(entry);
+    
+    // Put entry in targetMap
+    int characterId = CharacterDefinition.lookupCharacterClass(entry[0]);
+    addMapping(characterId, wordId);
+    return result;
+  }
+  
+  /**
+   * Put mapping from unicode code point to character class.
+   * 
+   * @param codePoint code point
+   * @param characterClassName character class name
+   */
+  public void putCharacterCategory(int codePoint, String characterClassName) {
+    characterDefinition.putCharacterCategory(codePoint, characterClassName);
+  }
+  
+  public void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
+    characterDefinition.putInvokeDefinition(characterClassName, invoke, group, length);
+  }
+  
+  @Override
+  public void write(String baseDir) throws IOException {
+    super.write(baseDir);
+    characterDefinition.write(baseDir);
+  }
+}