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:56 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 49098e522 -> 8651fbaa8


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/tools/test/org/apache/lucene/analysis/ko/dict/UnknownDictionaryTest.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/tools/test/org/apache/lucene/analysis/ko/dict/UnknownDictionaryTest.java b/lucene/analysis/nori/src/tools/test/org/apache/lucene/analysis/ko/dict/UnknownDictionaryTest.java
new file mode 100644
index 0000000..2bba714
--- /dev/null
+++ b/lucene/analysis/nori/src/tools/test/org/apache/lucene/analysis/ko/dict/UnknownDictionaryTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.util.CSVUtil;
+import org.apache.lucene.analysis.ko.util.UnknownDictionaryWriter;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+public class UnknownDictionaryTest extends LuceneTestCase {
+
+  @Test
+  public void testPutCharacterCategory() {
+    UnknownDictionaryWriter unkDic = new UnknownDictionaryWriter(10 * 1024 * 1024);
+    
+    try{
+      unkDic.putCharacterCategory(0, "DUMMY_NAME");
+      fail();
+    } catch(Exception e) {
+      
+    }
+    
+    try{
+      unkDic.putCharacterCategory(-1, "HANGUL");
+      fail();
+    } catch(Exception e) {
+      
+    }
+    
+    unkDic.putCharacterCategory(0, "DEFAULT");
+    unkDic.putCharacterCategory(1, "GREEK");
+    unkDic.putCharacterCategory(2, "HANJA");
+    unkDic.putCharacterCategory(3, "HANGUL");
+    unkDic.putCharacterCategory(4, "KANJI");
+  }
+  
+  @Test
+  public void testPut() {
+    UnknownDictionaryWriter unkDic = new UnknownDictionaryWriter(10 * 1024 * 1024);
+    try{
+      unkDic.put(CSVUtil.parse("HANGUL,1800,3562,UNKNOWN,*,*,*,*,*,*,*"));
+      fail();
+    } catch(Exception e){
+      
+    }
+
+    String entry1 = "ALPHA,1793,3533,795,SL,*,*,*,*,*,*,*";
+    String entry2 = "HANGUL,1800,3562,10247,UNKNOWN,*,*,*,*,*,*,*";
+    String entry3 = "HANJA,1792,3554,-821,SH,*,*,*,*,*,*,*";
+
+    unkDic.putCharacterCategory(0, "ALPHA");
+    unkDic.putCharacterCategory(1, "HANGUL");
+    unkDic.putCharacterCategory(2, "HANJA");
+    
+    unkDic.put(CSVUtil.parse(entry1));
+    unkDic.put(CSVUtil.parse(entry2));
+    unkDic.put(CSVUtil.parse(entry3));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index f19e33d..14e7194 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -85,6 +85,7 @@ io.prometheus.version = 0.2.0
 /junit/junit = 4.10
 
 /mecab/mecab-ipadic = 2.7.0-20070801
+/mecab/mecab-ko-dic = 2.0.3-20170922
 /mecab/mecab-naist-jdic = 0.6.3b-20111013
 /net.arnx/jsonic = 1.2.7
 /net.bytebuddy/byte-buddy = 1.6.2

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/module-build.xml
----------------------------------------------------------------------
diff --git a/lucene/module-build.xml b/lucene/module-build.xml
index 92ca60e..9f17c1c 100644
--- a/lucene/module-build.xml
+++ b/lucene/module-build.xml
@@ -451,6 +451,28 @@
     <property name="analyzers-morfologik-javadocs.uptodate" value="true"/>
   </target>
 
+  <property name="analyzers-nori.jar" value="${common.dir}/build/analysis/nori/lucene-analyzers-nori-${version}.jar"/>
+  <target name="check-analyzers-nori-uptodate" unless="analyzers-nori.uptodate">
+    <module-uptodate name="analysis/nori" jarfile="${analyzers-nori.jar}" property="analyzers-nori.uptodate"/>
+  </target>
+  <target name="jar-analyzers-nori" unless="analyzers-nori.uptodate" depends="check-analyzers-nori-uptodate">
+    <ant dir="${common.dir}/analysis/nori" target="jar-core" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="analyzers-nori.uptodate" value="true"/>
+  </target>
+
+  <property name="analyzers-nori-javadoc.jar" value="${common.dir}/build/analysis/nori/lucene-analyzers-nori-${version}-javadoc.jar"/>
+  <target name="check-analyzers-nori-javadocs-uptodate" unless="analyzers-nori-javadocs.uptodate">
+    <module-uptodate name="analysis/nori" jarfile="${analyzers-nori-javadoc.jar}" property="analyzers-nori-javadocs.uptodate"/>
+  </target>
+  <target name="javadocs-analyzers-nori" unless="analyzers-nori-javadocs.uptodate" depends="check-analyzers-nori-javadocs-uptodate">
+    <ant dir="${common.dir}/analysis/nori" target="javadocs" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+    <property name="analyzers-nori-javadocs.uptodate" value="true"/>
+  </target>
+
   <property name="codecs.jar" value="${common.dir}/build/codecs/lucene-codecs-${version}.jar"/>
   <target name="check-codecs-uptodate" unless="codecs.uptodate">
     <module-uptodate name="codecs" jarfile="${codecs.jar}" property="codecs.uptodate"/>


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
new file mode 100644
index 0000000..43a02d3
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizerFactory.java
@@ -0,0 +1,89 @@
+/*
+ * 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.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.lucene.analysis.ko.dict.UserDictionary;
+import org.apache.lucene.analysis.util.TokenizerFactory;
+import org.apache.lucene.util.AttributeFactory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+
+/**
+ * Factory for {@link KoreanTokenizer}.
+ * @lucene.experimental
+ */
+public class KoreanTokenizerFactory extends TokenizerFactory implements ResourceLoaderAware {
+  private static final String USER_DICT_PATH = "userDictionary";
+  private static final String USER_DICT_ENCODING = "userDictionaryEncoding";
+  private static final String DECOMPOUND_MODE = "decompoundMode";
+  private static final String OUTPUT_UNKNOWN_UNIGRAMS = "outputUnknownUnigrams";
+
+  private final String userDictionaryPath;
+  private final String userDictionaryEncoding;
+  private UserDictionary userDictionary;
+
+  private final KoreanTokenizer.DecompoundMode mode;
+  private final boolean outputUnknownUnigrams;
+
+  /** Creates a new KoreanTokenizerFactory */
+  public KoreanTokenizerFactory(Map<String, String> args) {
+    super(args);
+    userDictionaryPath = args.remove(USER_DICT_PATH);
+    userDictionaryEncoding = args.remove(USER_DICT_ENCODING);
+    mode = KoreanTokenizer.DecompoundMode.valueOf(get(args, DECOMPOUND_MODE, KoreanTokenizer.DEFAULT_DECOMPOUND.toString()).toUpperCase(Locale.ROOT));
+    outputUnknownUnigrams = getBoolean(args, OUTPUT_UNKNOWN_UNIGRAMS, false);
+
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    if (userDictionaryPath != null) {
+      try (InputStream stream = loader.openResource(userDictionaryPath)) {
+        String encoding = userDictionaryEncoding;
+        if (encoding == null) {
+          encoding = IOUtils.UTF_8;
+        }
+        CharsetDecoder decoder = Charset.forName(encoding).newDecoder()
+          .onMalformedInput(CodingErrorAction.REPORT)
+          .onUnmappableCharacter(CodingErrorAction.REPORT);
+        Reader reader = new InputStreamReader(stream, decoder);
+        userDictionary = UserDictionary.open(reader);
+      }
+    } else {
+      userDictionary = null;
+    }
+  }
+
+  @Override
+  public KoreanTokenizer create(AttributeFactory factory) {
+    return new KoreanTokenizer(factory, userDictionary, mode, outputUnknownUnigrams);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/POS.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/POS.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/POS.java
new file mode 100644
index 0000000..263c9c8
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/POS.java
@@ -0,0 +1,304 @@
+/*
+ * 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.util.Locale;
+
+/**
+ * Part of speech classification for Korean based on Sejong corpus classification.
+ * The list of tags and their meanings is available here:
+ * https://docs.google.com/spreadsheets/d/1-9blXKjtjeKZqsf4NzHeYJCrr49-nXeRF6D80udfcwY
+ */
+public class POS {
+
+  /**
+   * The type of the token.
+   */
+  public enum Type {
+    /**
+     * A simple morpheme.
+     */
+    MORPHEME,
+
+    /**
+     * Compound noun.
+     */
+    COMPOUND,
+
+    /**
+     * Inflected token.
+     */
+    INFLECT,
+
+    /**
+     * Pre-analysis token.
+     */
+    PREANALYSIS,
+  }
+
+  /**
+   * Part of speech tag for Korean based on Sejong corpus classification.
+   */
+  public enum Tag {
+    /**
+     * Verbal endings
+     */
+    E(100, "Verbal endings"),
+
+    /**
+     * Interjection
+     */
+    IC(110, "Interjection"),
+
+    /**
+     * Ending Particle
+     */
+    J(120, "Ending Particle"),
+
+    /**
+     * General Adverb
+     */
+    MAG(130, "General Adverb"),
+
+    /**
+     * Conjunctive adverb
+     */
+    MAJ(131, "Conjunctive adverb"),
+
+    /**
+     * Determiner
+     **/
+    MM(140, "Modifier"),
+
+    /**
+     * General Noun
+     **/
+    NNG(150, "General Noun"),
+
+    /**
+     * Proper Noun
+     **/
+    NNP(151, "Proper Noun"),
+
+    /**
+     * Dependent noun (following nouns)
+     **/
+    NNB(152, "Dependent noun"),
+
+    /**
+     * Dependent noun
+     **/
+    NNBC(153, "Dependent noun"),
+
+    /**
+     * Pronoun
+     **/
+    NP(154, "Pronoun"),
+
+    /**
+     * Numeral
+     **/
+    NR(155, "Numeral"),
+
+    /**
+     * Terminal punctuation (? ! .)
+     **/
+    SF(160, "Terminal punctuation"),
+
+    /**
+     * Chinese character
+     **/
+    SH(161, "Chinese Characeter"),
+
+    /**
+     * Foreign language
+     **/
+    SL(162, "Foreign language"),
+
+    /**
+     * Number
+     **/
+    SN(163, "Number"),
+
+    /**
+     * Space
+     **/
+    SP(164, "Space"),
+
+    /**
+     * Closing brackets
+     **/
+    SSC(165, "Closing brackets"),
+
+    /**
+     * Opening brackets
+     **/
+    SSO(166, "Opening brackets"),
+
+    /**
+     * Separator (· / :)
+     **/
+    SC(167, "Separator"),
+
+    /**
+     * Other symbol
+     **/
+    SY(168, "Other symbol"),
+
+    /**
+     * Ellipsis
+     **/
+    SE(169, "Ellipsis"),
+
+    /**
+     * Adjective
+     **/
+    VA(170, "Adjective"),
+
+    /**
+     * Negative designator
+     **/
+    VCN(171, "Negative designator"),
+
+    /**
+     * Positive designator
+     **/
+    VCP(172, "Positive designator"),
+
+    /**
+     * Verb
+     **/
+    VV(173, "Verb"),
+
+    /**
+     * Auxiliary Verb or Adjective
+     **/
+    VX(174, "Auxiliary Verb or Adjective"),
+
+    /**
+     * Prefix
+     **/
+    XPN(181, "Prefix"),
+
+    /**
+     * Root
+     **/
+    XR(182, "Root"),
+
+    /**
+     * Adjective Suffix
+     **/
+    XSA(183, "Adjective Suffix"),
+
+    /**
+     * Noun Suffix
+     **/
+    XSN(184, "Noun Suffix"),
+
+    /**
+     * Verb Suffix
+     **/
+    XSV(185, "Verb Suffix"),
+
+    /**
+     * Unknown
+     */
+    UNKNOWN(999, "Unknown"),
+
+    /**
+     * Unknown
+     */
+    UNA(-1, "Unknown"),
+
+    /**
+     * Unknown
+     */
+    NA(-1, "Unknown"),
+
+    /**
+     * Unknown
+     */
+    VSV(-1, "Unknown");
+
+    private final int code;
+    private final String desc;
+
+    /**
+     * Returns the code associated with the tag (as defined in pos-id.def).
+     */
+    public int code() {
+      return code;
+    }
+
+    /**
+     * Returns the description associated with the tag.
+     */
+    public String description() {
+      return desc;
+    }
+
+    /**
+     * Returns a new part of speech tag.
+     * @param code The code for the tag.
+     * @param desc The description of the tag.
+     */
+    Tag(int code, String desc) {
+      this.code = code;
+      this.desc = desc;
+    }
+  }
+
+  /**
+   * Returns the {@link Tag} of the provided <code>name</code>.
+   */
+  public static Tag resolveTag(String name) {
+    String tagUpper = name.toUpperCase(Locale.ENGLISH);
+    if (tagUpper.startsWith("J")) {
+      return Tag.J;
+    } else if (tagUpper.startsWith("E")) {
+      return Tag.E;
+    } else {
+      return Tag.valueOf(tagUpper);
+    }
+  }
+
+  /**
+   * Returns the {@link Tag} of the provided <code>tag</code>.
+   */
+  public static Tag resolveTag(byte tag) {
+    assert tag < Tag.values().length;
+    return Tag.values()[tag];
+  }
+
+  /**
+   * Returns the {@link Type} of the provided <code>name</code>.
+   */
+  public static Type resolveType(String name) {
+    if ("*".equals(name)) {
+      return Type.MORPHEME;
+    }
+    return Type.valueOf(name.toUpperCase(Locale.ENGLISH));
+  }
+
+  /**
+   * Returns the {@link Type} of the provided <code>type</code>.
+   */
+  public static Type resolveType(byte type) {
+    assert type < Type.values().length;
+    return Type.values()[type];
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/Token.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/Token.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/Token.java
new file mode 100644
index 0000000..bf2c528
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/Token.java
@@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.lucene.analysis.ko.dict.Dictionary.Morpheme;
+
+/**
+ * Analyzed token with morphological data.
+ */
+public abstract class Token {
+  private final char[] surfaceForm;
+  private final int offset;
+  private final int length;
+
+  private final int startOffset;
+  private final int endOffset;
+  private int posIncr = 1;
+  private int posLen = 1;
+
+  public Token(char[] surfaceForm, int offset, int length, int startOffset, int endOffset) {
+    this.surfaceForm = surfaceForm;
+    this.offset = offset;
+    this.length = length;
+
+    this.startOffset = startOffset;
+    this.endOffset = endOffset;
+  }
+
+  /**
+   * @return surfaceForm
+   */
+  public char[] getSurfaceForm() {
+    return surfaceForm;
+  }
+
+  /**
+   * @return offset into surfaceForm
+   */
+  public int getOffset() {
+    return offset;
+  }
+
+  /**
+   * @return length of surfaceForm
+   */
+  public int getLength() {
+    return length;
+  }
+
+  /**
+   * @return surfaceForm as a String
+   */
+  public String getSurfaceFormString() {
+    return new String(surfaceForm, offset, length);
+  }
+
+  /**
+   * Get the {@link POS.Type} of the token.
+   */
+  public abstract POS.Type getPOSType();
+
+  /**
+   * Get the left part of speech of the token.
+   */
+  public abstract POS.Tag getLeftPOS();
+
+  /**
+   * Get the right part of speech of the token.
+   */
+  public abstract POS.Tag getRightPOS();
+
+  /**
+   * Get the reading of the token.
+   */
+  public abstract String getReading();
+
+  /**
+   * Get the {@link Morpheme} decomposition of the token.
+   */
+  public abstract Morpheme[] getMorphemes();
+
+  /**
+   * Get the start offset of the term in the analyzed text.
+   */
+  public int getStartOffset() {
+    return startOffset;
+  }
+
+  /**
+   * Get the end offset of the term in the analyzed text.
+   */
+  public int getEndOffset() {
+    return endOffset;
+  }
+
+  public void setPositionIncrement(int posIncr) {
+    this.posIncr = posIncr;
+  }
+
+  public int getPositionIncrement() {
+    return posIncr;
+  }
+
+  public void setPositionLength(int posLen) {
+    this.posLen = posLen;
+  }
+
+  public int getPositionLength() {
+    return posLen;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/BinaryDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/BinaryDictionary.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/BinaryDictionary.java
new file mode 100644
index 0000000..b7a3612
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/BinaryDictionary.java
@@ -0,0 +1,239 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+
+/**
+ * Base class for a binary-encoded in-memory dictionary.
+ */
+public abstract class BinaryDictionary implements Dictionary {
+  public static final String TARGETMAP_FILENAME_SUFFIX = "$targetMap.dat";
+  public static final String DICT_FILENAME_SUFFIX = "$buffer.dat";
+  public static final String POSDICT_FILENAME_SUFFIX = "$posDict.dat";
+
+  public static final String DICT_HEADER = "ko_dict";
+  public static final String TARGETMAP_HEADER = "ko_dict_map";
+  public static final String POSDICT_HEADER = "ko_dict_pos";
+  public static final int VERSION = 1;
+
+  private final ByteBuffer buffer;
+  private final int[] targetMapOffsets, targetMap;
+  private final POS.Tag[] posDict;
+
+  protected BinaryDictionary() throws IOException {
+    InputStream mapIS = null, dictIS = null, posIS = null;
+    int[] targetMapOffsets = null, targetMap = null;
+    ByteBuffer buffer = null;
+    boolean success = false;
+    try {
+      mapIS = getResource(TARGETMAP_FILENAME_SUFFIX);
+      mapIS = new BufferedInputStream(mapIS);
+      DataInput in = new InputStreamDataInput(mapIS);
+      CodecUtil.checkHeader(in, TARGETMAP_HEADER, VERSION, VERSION);
+      targetMap = new int[in.readVInt()];
+      targetMapOffsets = new int[in.readVInt()];
+      int accum = 0, sourceId = 0;
+      for (int ofs = 0; ofs < targetMap.length; ofs++) {
+        final int val = in.readVInt();
+        if ((val & 0x01) != 0) {
+          targetMapOffsets[sourceId] = ofs;
+          sourceId++;
+        }
+        accum += val >>> 1;
+        targetMap[ofs] = accum;
+      }
+      if (sourceId + 1 != targetMapOffsets.length)
+        throw new IOException("targetMap file format broken");
+      targetMapOffsets[sourceId] = targetMap.length;
+      mapIS.close(); mapIS = null;
+
+      posIS = getResource(POSDICT_FILENAME_SUFFIX);
+      posIS = new BufferedInputStream(posIS);
+      in = new InputStreamDataInput(posIS);
+      CodecUtil.checkHeader(in, POSDICT_HEADER, VERSION, VERSION);
+      int posSize = in.readVInt();
+      posDict = new POS.Tag[posSize];
+      for (int j = 0; j < posSize; j++) {
+        posDict[j] = POS.resolveTag(in.readByte());
+      }
+      posIS.close(); posIS = null;
+
+      dictIS = getResource(DICT_FILENAME_SUFFIX);
+      // no buffering here, as we load in one large buffer
+      in = new InputStreamDataInput(dictIS);
+      CodecUtil.checkHeader(in, DICT_HEADER, VERSION, VERSION);
+      final int size = in.readVInt();
+      final ByteBuffer tmpBuffer = ByteBuffer.allocateDirect(size);
+      final ReadableByteChannel channel = Channels.newChannel(dictIS);
+      final int read = channel.read(tmpBuffer);
+      if (read != size) {
+        throw new EOFException("Cannot read whole dictionary");
+      }
+      dictIS.close(); dictIS = null;
+      buffer = tmpBuffer.asReadOnlyBuffer();
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(mapIS, dictIS);
+      } else {
+        IOUtils.closeWhileHandlingException(mapIS, dictIS);
+      }
+    }
+
+    this.targetMap = targetMap;
+    this.targetMapOffsets = targetMapOffsets;
+    this.buffer = buffer;
+  }
+  
+  protected final InputStream getResource(String suffix) throws IOException {
+    return getClassResource(getClass(), suffix);
+  }
+  
+  // util, reused by ConnectionCosts and CharacterDefinition
+  public static final InputStream getClassResource(Class<?> clazz, String suffix) throws IOException {
+    final InputStream is = clazz.getResourceAsStream(clazz.getSimpleName() + suffix);
+    if (is == null)
+      throw new FileNotFoundException("Not in classpath: " + clazz.getName().replace('.','/') + suffix);
+    return is;
+  }
+
+  public void lookupWordIds(int sourceId, IntsRef ref) {
+    ref.ints = targetMap;
+    ref.offset = targetMapOffsets[sourceId];
+    // targetMapOffsets always has one more entry pointing behind last:
+    ref.length = targetMapOffsets[sourceId + 1] - ref.offset;
+  }
+
+  @Override
+  public int getLeftId(int wordId) {
+    return buffer.getShort(wordId) >>> 2;
+  }
+  
+  @Override
+  public int getRightId(int wordId) {
+    return buffer.getShort(wordId+2) >>> 2; // Skip left id
+  }
+  
+  @Override
+  public int getWordCost(int wordId) {
+    return buffer.getShort(wordId + 4);  // Skip left and right id
+  }
+
+  @Override
+  public POS.Type getPOSType(int wordId) {
+    byte value = (byte) (buffer.getShort(wordId) & 3);
+    return POS.resolveType(value);
+  }
+
+  @Override
+  public POS.Tag getLeftPOS(int wordId) {
+    return posDict[getLeftId(wordId)];
+  }
+
+  @Override
+  public POS.Tag getRightPOS(int wordId) {
+    POS.Type type = getPOSType(wordId);
+    if (type == POS.Type.MORPHEME || type == POS.Type.COMPOUND || hasSinglePOS(wordId)) {
+      return getLeftPOS(wordId);
+    } else {
+      byte value = buffer.get(wordId + 6);
+      return POS.resolveTag(value);
+    }
+  }
+
+  @Override
+  public String getReading(int wordId) {
+    if (hasReadingData(wordId)) {
+      int offset = wordId + 6;
+      return readString(offset);
+    }
+    return null;
+  }
+
+  @Override
+  public Morpheme[] getMorphemes(int wordId, char[] surfaceForm, int off, int len) {
+    POS.Type posType = getPOSType(wordId);
+    if (posType == POS.Type.MORPHEME) {
+      return null;
+    }
+    int offset = wordId + 6;
+    boolean hasSinglePos = hasSinglePOS(wordId);
+    if (hasSinglePos == false) {
+      offset ++; // skip rightPOS
+    }
+    int length = buffer.get(offset++);
+    if (length == 0) {
+      return null;
+    }
+    Morpheme[] morphemes = new Morpheme[length];
+    int surfaceOffset = 0;
+    final POS.Tag leftPOS = getLeftPOS(wordId);
+    for (int i = 0; i < length; i++) {
+      final String form;
+      final POS.Tag tag = hasSinglePos ? leftPOS : POS.resolveTag(buffer.get(offset++));
+      if (posType == POS.Type.INFLECT) {
+        form = readString(offset);
+        offset += form.length() * 2 + 1;
+      } else {
+        int formLen = buffer.get(offset++);
+        form = new String(surfaceForm, off+surfaceOffset, formLen);
+        surfaceOffset += formLen;
+      }
+      morphemes[i] = new Morpheme(tag, form);
+    }
+    return morphemes;
+  }
+
+  private String readString(int offset) {
+    int strOffset = offset;
+    int len = buffer.get(strOffset++);
+    char text[] = new char[len];
+    for (int i = 0; i < len; i++) {
+      text[i] = buffer.getChar(strOffset + (i<<1));
+    }
+    return new String(text);
+  }
+
+  private boolean hasSinglePOS(int wordId) {
+    return (buffer.getShort(wordId+2) & HAS_SINGLE_POS) != 0;
+  }
+
+  private boolean hasReadingData(int wordId) {
+    return (buffer.getShort(wordId+2) & HAS_READING) != 0;
+  }
+
+  /** flag that the entry has a single part of speech (leftPOS) */
+  public static final int HAS_SINGLE_POS = 1;
+
+  /** flag that the entry has reading data. otherwise reading is surface form */
+  public static final int HAS_READING = 2;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/CharacterDefinition.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/CharacterDefinition.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/CharacterDefinition.java
new file mode 100644
index 0000000..bc81cba
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/CharacterDefinition.java
@@ -0,0 +1,136 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Character category data.
+ */
+public final class CharacterDefinition {
+
+  public static final String FILENAME_SUFFIX = ".dat";
+  public static final String HEADER = "ko_cd";
+  public static final int VERSION = 1;
+
+  public static final int CLASS_COUNT = CharacterClass.values().length;
+
+  // only used internally for lookup:
+  private enum CharacterClass {
+    NGRAM, DEFAULT, SPACE, SYMBOL, NUMERIC, ALPHA, CYRILLIC, GREEK, HIRAGANA, KATAKANA, KANJI, HANGUL, HANJA, HANJANUMERIC;
+  }
+
+  private final byte[] characterCategoryMap = new byte[0x10000];
+
+  private final boolean[] invokeMap = new boolean[CLASS_COUNT];
+  private final boolean[] groupMap = new boolean[CLASS_COUNT];
+
+  // the classes:
+  public static final byte NGRAM = (byte) CharacterClass.NGRAM.ordinal();
+  public static final byte DEFAULT = (byte) CharacterClass.DEFAULT.ordinal();
+  public static final byte SPACE = (byte) CharacterClass.SPACE.ordinal();
+  public static final byte SYMBOL = (byte) CharacterClass.SYMBOL.ordinal();
+  public static final byte NUMERIC = (byte) CharacterClass.NUMERIC.ordinal();
+  public static final byte ALPHA = (byte) CharacterClass.ALPHA.ordinal();
+  public static final byte CYRILLIC = (byte) CharacterClass.CYRILLIC.ordinal();
+  public static final byte GREEK = (byte) CharacterClass.GREEK.ordinal();
+  public static final byte HIRAGANA = (byte) CharacterClass.HIRAGANA.ordinal();
+  public static final byte KATAKANA = (byte) CharacterClass.KATAKANA.ordinal();
+  public static final byte KANJI = (byte) CharacterClass.KANJI.ordinal();
+  public static final byte HANGUL = (byte) CharacterClass.HANGUL.ordinal();
+  public static final byte HANJA = (byte) CharacterClass.HANJA.ordinal();
+  public static final byte HANJANUMERIC = (byte) CharacterClass.HANJANUMERIC.ordinal();
+  
+  private CharacterDefinition() throws IOException {
+    InputStream is = null;
+    boolean success = false;
+    try {
+      is = BinaryDictionary.getClassResource(getClass(), FILENAME_SUFFIX);
+      is = new BufferedInputStream(is);
+      final DataInput in = new InputStreamDataInput(is);
+      CodecUtil.checkHeader(in, HEADER, VERSION, VERSION);
+      in.readBytes(characterCategoryMap, 0, characterCategoryMap.length);
+      for (int i = 0; i < CLASS_COUNT; i++) {
+        final byte b = in.readByte();
+        invokeMap[i] = (b & 0x01) != 0;
+        groupMap[i] = (b & 0x02) != 0;
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(is);
+      } else {
+        IOUtils.closeWhileHandlingException(is);
+      }
+    }
+  }
+  
+  public byte getCharacterClass(char c) {
+    return characterCategoryMap[c];
+  }
+  
+  public boolean isInvoke(char c) {
+    return invokeMap[characterCategoryMap[c]];
+  }
+  
+  public boolean isGroup(char c) {
+    return groupMap[characterCategoryMap[c]];
+  }
+
+  public boolean isHanja(char c) {
+    final byte characterClass = getCharacterClass(c);
+    return characterClass == HANJA || characterClass == HANJANUMERIC;
+  }
+
+  public boolean isHangul(char c) {
+    return getCharacterClass(c) == HANGUL;
+  }
+
+  public boolean hasCoda(char ch){
+    if (((ch - 0xAC00) % 0x001C) == 0) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  public static byte lookupCharacterClass(String characterClassName) {
+    return (byte) CharacterClass.valueOf(characterClassName).ordinal();
+  }
+
+  public static CharacterDefinition getInstance() {
+    return SingletonHolder.INSTANCE;
+  }
+  
+  private static class SingletonHolder {
+    static final CharacterDefinition INSTANCE;
+    static {
+      try {
+        INSTANCE = new CharacterDefinition();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Cannot load CharacterDefinition.", ioe);
+      }
+    }
+   }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/ConnectionCosts.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/ConnectionCosts.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/ConnectionCosts.java
new file mode 100644
index 0000000..95d0e8b
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/ConnectionCosts.java
@@ -0,0 +1,96 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * n-gram connection cost data
+ */
+public final class ConnectionCosts {
+  
+  public static final String FILENAME_SUFFIX = ".dat";
+  public static final String HEADER = "ko_cc";
+  public static final int VERSION = 1;
+
+  private final ByteBuffer buffer;
+  private final int forwardSize;
+
+  private ConnectionCosts() throws IOException {
+    InputStream is = null;
+    ByteBuffer buffer = null;
+    boolean success = false;
+    try {
+      is = BinaryDictionary.getClassResource(getClass(), FILENAME_SUFFIX);
+      is = new BufferedInputStream(is);
+      final DataInput in = new InputStreamDataInput(is);
+      CodecUtil.checkHeader(in, HEADER, VERSION, VERSION);
+      this.forwardSize = in.readVInt();
+      int backwardSize = in.readVInt();
+      int size = forwardSize * backwardSize;
+
+      // copy the matrix into a direct byte buffer
+      final ByteBuffer tmpBuffer = ByteBuffer.allocateDirect(size*2);
+      int accum = 0;
+      for (int j = 0; j < backwardSize; j++) {
+        for (int i = 0; i < forwardSize; i++) {
+          accum += in.readZInt();
+          tmpBuffer.putShort((short) accum);
+        }
+      }
+      buffer = tmpBuffer.asReadOnlyBuffer();
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(is);
+      } else {
+        IOUtils.closeWhileHandlingException(is);
+      }
+    }
+    this.buffer = buffer;
+  }
+  
+  public int get(int forwardId, int backwardId) {
+    // map 2d matrix into a single dimension short array
+    int offset = (backwardId * forwardSize + forwardId) * 2;
+    return buffer.getShort(offset);
+  }
+  
+  public static ConnectionCosts getInstance() {
+    return SingletonHolder.INSTANCE;
+  }
+  
+  private static class SingletonHolder {
+    static final ConnectionCosts INSTANCE;
+    static {
+      try {
+        INSTANCE = new ConnectionCosts();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Cannot load ConnectionCosts.", ioe);
+      }
+    }
+   }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/Dictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/Dictionary.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/Dictionary.java
new file mode 100644
index 0000000..23101b3
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/Dictionary.java
@@ -0,0 +1,83 @@
+/*
+ * 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.Tag;
+import org.apache.lucene.analysis.ko.POS.Type;
+
+/**
+ * Dictionary interface for retrieving morphological data
+ * by id.
+ */
+public interface Dictionary {
+  /**
+   * A morpheme extracted from a compound token.
+   */
+  class Morpheme {
+    public final Tag posTag;
+    public final String surfaceForm;
+
+    public Morpheme(Tag posTag, String surfaceForm) {
+      this.posTag = posTag;
+      this.surfaceForm = surfaceForm;
+    }
+  }
+
+  /**
+   * Get left id of specified word
+   */
+  int getLeftId(int wordId);
+  
+  /**
+   * Get right id of specified word
+   */
+  int getRightId(int wordId);
+  
+  /**
+   * Get word cost of specified word
+   */
+  int getWordCost(int wordId);
+
+  /**
+   * Get the {@link Type} of specified word (morpheme, compound, inflect or pre-analysis)
+   */
+  Type getPOSType(int wordId);
+
+  /**
+   * Get the left {@link Tag} of specfied word.
+   *
+   * For {@link Type#MORPHEME} and {@link Type#COMPOUND} the left and right POS are the same.
+   */
+  Tag getLeftPOS(int wordId);
+
+  /**
+   * Get the right {@link Tag} of specfied word.
+   *
+   * For {@link Type#MORPHEME} and {@link Type#COMPOUND} the left and right POS are the same.
+   */
+  Tag getRightPOS(int wordId);
+
+  /**
+   * Get the reading of specified word (mainly used for Hanja to Hangul conversion).
+   */
+  String getReading(int wordId);
+
+  /**
+   * Get the morphemes of specified word (e.g. 가깝으나: 가깝 + 으나).
+   */
+  Morpheme[] getMorphemes(int wordId, char[] surfaceForm, int off, int len);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary.java
new file mode 100644
index 0000000..94408c7
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary.java
@@ -0,0 +1,77 @@
+/*
+ * 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.BufferedInputStream;
+import java.io.InputStream;
+import java.io.IOException;
+
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+
+/**
+ * Binary dictionary implementation for a known-word dictionary model:
+ * Words are encoded into an FST mapping to a list of wordIDs.
+ */
+public final class TokenInfoDictionary extends BinaryDictionary {
+
+  public static final String FST_FILENAME_SUFFIX = "$fst.dat";
+
+  private final TokenInfoFST fst;
+  
+  private TokenInfoDictionary() throws IOException {
+    super();
+    InputStream is = null;
+    FST<Long> fst = null;
+    boolean success = false;
+    try {
+      is = getResource(FST_FILENAME_SUFFIX);
+      is = new BufferedInputStream(is);
+      fst = new FST<>(new InputStreamDataInput(is), PositiveIntOutputs.getSingleton());
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(is);
+      } else {
+        IOUtils.closeWhileHandlingException(is);
+      }
+    }
+    this.fst = new TokenInfoFST(fst);
+  }
+  
+  public TokenInfoFST getFST() {
+    return fst;
+  }
+   
+  public static TokenInfoDictionary getInstance() {
+    return SingletonHolder.INSTANCE;
+  }
+  
+  private static class SingletonHolder {
+    static final TokenInfoDictionary INSTANCE;
+    static {
+      try {
+        INSTANCE = new TokenInfoDictionary();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Cannot load TokenInfoDictionary.", ioe);
+      }
+    }
+   }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoFST.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoFST.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoFST.java
new file mode 100644
index 0000000..7f9bec6
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/TokenInfoFST.java
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST;
+
+/**
+ * Thin wrapper around an FST with root-arc caching for Hangul syllables (11,172 arcs).
+ */
+public final class TokenInfoFST {
+  private final FST<Long> fst;
+
+  private final int cacheCeiling;
+  private final FST.Arc<Long> rootCache[];
+  
+  public final Long NO_OUTPUT;
+
+  public TokenInfoFST(FST<Long> fst) throws IOException {
+    this.fst = fst;
+    this.cacheCeiling = 0xD7A3;
+    NO_OUTPUT = fst.outputs.getNoOutput();
+    rootCache = cacheRootArcs();
+  }
+  
+  @SuppressWarnings({"rawtypes","unchecked"})
+  private FST.Arc<Long>[] cacheRootArcs() throws IOException {
+    FST.Arc<Long> rootCache[] = new FST.Arc[1+(cacheCeiling-0xAC00)];
+    FST.Arc<Long> firstArc = new FST.Arc<>();
+    fst.getFirstArc(firstArc);
+    FST.Arc<Long> arc = new FST.Arc<>();
+    final FST.BytesReader fstReader = fst.getBytesReader();
+    // TODO: jump to AC00, readNextRealArc to ceiling? (just be careful we don't add bugs)
+    for (int i = 0; i < rootCache.length; i++) {
+      if (fst.findTargetArc(0xAC00 + i, firstArc, arc, fstReader) != null) {
+        rootCache[i] = new FST.Arc<Long>().copyFrom(arc);
+      }
+    }
+    return rootCache;
+  }
+
+  public FST.Arc<Long> findTargetArc(int ch, FST.Arc<Long> follow, FST.Arc<Long> arc, boolean useCache, FST.BytesReader fstReader) throws IOException {
+    if (useCache && ch >= 0xAC00 && ch <= cacheCeiling) {
+      assert ch != FST.END_LABEL;
+      final Arc<Long> result = rootCache[ch - 0xAC00];
+      if (result == null) {
+        return null;
+      } else {
+        arc.copyFrom(result);
+        return arc;
+      }
+    } else {
+      return fst.findTargetArc(ch, follow, arc, fstReader);
+    }
+  }
+  
+  public Arc<Long> getFirstArc(FST.Arc<Long> arc) {
+    return fst.getFirstArc(arc);
+  }
+
+  public FST.BytesReader getBytesReader() {
+    return fst.getBytesReader();
+  }
+
+  /** @lucene.internal for testing only */
+  FST<Long> getInternalFST() {
+    return fst;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UnknownDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UnknownDictionary.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UnknownDictionary.java
new file mode 100644
index 0000000..6d56b92
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UnknownDictionary.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Dictionary for unknown-word handling.
+ */
+public final class UnknownDictionary extends BinaryDictionary {
+  private final CharacterDefinition characterDefinition = CharacterDefinition.getInstance();
+
+  private UnknownDictionary() throws IOException {
+    super();
+  }
+
+  public CharacterDefinition getCharacterDefinition() {
+    return characterDefinition;
+  }
+
+  public static UnknownDictionary getInstance() {
+    return SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  public String getReading(int wordId) {
+    return null;
+  }
+
+  @Override
+  public Morpheme[] getMorphemes(int wordId, char[] surfaceForm, int off, int len) {
+    return null;
+  }
+
+  private static class SingletonHolder {
+    static final UnknownDictionary INSTANCE;
+
+    static {
+      try {
+        INSTANCE = new UnknownDictionary();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Cannot load UnknownDictionary.", ioe);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UserDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UserDictionary.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UserDictionary.java
new file mode 100644
index 0000000..c5378a9
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/UserDictionary.java
@@ -0,0 +1,235 @@
+/*
+ * 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.BufferedReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.analysis.ko.POS;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+
+/**
+ * Class for building a User Dictionary.
+ * This class allows for adding custom nouns (세종) or compounds (세종시 세종 시).
+ */
+public final class UserDictionary implements Dictionary {
+  // text -> wordID
+  private final TokenInfoFST fst;
+
+  public static final int WORD_COST = -100000;
+
+  // NNG left
+  public static final short LEFT_ID = 1781;
+
+  // NNG right
+  public static final short RIGHT_ID = 3534;
+  // NNG right with hangul and a coda on the last char
+  public static final short RIGHT_ID_T = 3534;
+  // NNG right with hangul and no coda on the last char
+  public static final short RIGHT_ID_F = 3535;
+
+  // length, length... indexed by compound ID or null for simple noun
+  private final int segmentations[][];
+  private final short[] rightIds;
+
+  public static UserDictionary open(Reader reader) throws IOException {
+
+    BufferedReader br = new BufferedReader(reader);
+    String line = null;
+    List<String> entries = new ArrayList<>();
+
+    // text + optional segmentations
+    while ((line = br.readLine()) != null) {
+      // Remove comments
+      line = line.replaceAll("#.*$", "");
+
+      // Skip empty lines or comment lines
+      if (line.trim().length() == 0) {
+        continue;
+      }
+      entries.add(line);
+    }
+
+    if (entries.isEmpty()) {
+      return null;
+    } else {
+      return new UserDictionary(entries);
+    }
+  }
+
+  private UserDictionary(List<String> entries) throws IOException {
+    final CharacterDefinition charDef = CharacterDefinition.getInstance();
+    Collections.sort(entries,
+        Comparator.comparing(e -> e.split("\\s+")[0]));
+
+    PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton();
+    Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, fstOutput);
+    IntsRefBuilder scratch = new IntsRefBuilder();
+
+    String lastToken = null;
+    List<int[]> segmentations = new ArrayList<>(entries.size());
+    List<Short> rightIds = new ArrayList<>(entries.size());
+    long ord = 0;
+    for (String entry : entries) {
+      String[] splits = entry.split("\\s+");
+      String token = splits[0];
+      if (lastToken != null && token.equals(lastToken)) {
+        continue;
+      }
+      char lastChar = entry.charAt(entry.length()-1);
+      if (charDef.isHangul(lastChar)) {
+        if (charDef.hasCoda(lastChar)) {
+          rightIds.add(RIGHT_ID_T);
+        } else {
+          rightIds.add(RIGHT_ID_F);
+        }
+      } else {
+        rightIds.add(RIGHT_ID);
+      }
+
+      if (splits.length == 1) {
+        segmentations.add(null);
+      } else {
+        int[] length = new int[splits.length-1];
+        int offset = 0;
+        for (int i = 1; i < splits.length; i++) {
+          length[i-1] = splits[i].length();
+          offset += splits[i].length();
+        }
+        if (offset > token.length()) {
+          throw new IllegalArgumentException("Illegal user dictionary entry " + entry +
+              " - the segmentation is bigger than the surface form (" + token + ")");
+        }
+        segmentations.add(length);
+      }
+
+      // add mapping to FST
+      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);
+      lastToken = token;
+      ord ++;
+    }
+    this.fst = new TokenInfoFST(fstBuilder.finish());
+    this.segmentations = segmentations.toArray(new int[segmentations.size()][]);
+    this.rightIds = new short[rightIds.size()];
+    for (int i = 0; i < rightIds.size(); i++) {
+      this.rightIds[i] = rightIds.get(i);
+    }
+  }
+
+  public TokenInfoFST getFST() {
+    return fst;
+  }
+
+  @Override
+  public int getLeftId(int wordId) {
+    return LEFT_ID;
+  }
+  
+  @Override
+  public int getRightId(int wordId) {
+    return rightIds[wordId];
+  }
+  
+  @Override
+  public int getWordCost(int wordId) {
+    return WORD_COST;
+  }
+
+  @Override
+  public POS.Type getPOSType(int wordId) {
+    if (segmentations[wordId] == null) {
+      return POS.Type.MORPHEME;
+    } else {
+      return POS.Type.COMPOUND;
+    }
+  }
+
+  @Override
+  public POS.Tag getLeftPOS(int wordId) {
+    return POS.Tag.NNG;
+  }
+
+  @Override
+  public POS.Tag getRightPOS(int wordId) {
+    return POS.Tag.NNG;
+  }
+
+  @Override
+  public String getReading(int wordId) {
+    return null;
+  }
+
+  @Override
+  public Morpheme[] getMorphemes(int wordId, char[] surfaceForm, int off, int len) {
+    int[] segs = segmentations[wordId];
+    if (segs == null) {
+      return null;
+    }
+    int offset = 0;
+    Morpheme[] morphemes = new Morpheme[segs.length];
+    for (int i = 0; i < segs.length; i++) {
+      morphemes[i] = new Morpheme(POS.Tag.NNG, new String(surfaceForm, off+offset, segs[i]));
+      offset += segs[i];
+    }
+    return morphemes;
+  }
+
+  /**
+   * Lookup words in text
+   * @param chars text
+   * @param off offset into text
+   * @param len length of text
+   * @return array of wordId
+   */
+  public List<Integer> lookup(char[] chars, int off, int len) throws IOException {
+    List<Integer> result = new ArrayList<>();
+    final FST.BytesReader fstReader = fst.getBytesReader();
+
+    FST.Arc<Long> arc = new FST.Arc<>();
+    int end = off + len;
+    for (int startOffset = off; startOffset < end; startOffset++) {
+      arc = fst.getFirstArc(arc);
+      int output = 0;
+      int remaining = end - startOffset;
+      for (int i = 0; i < remaining; i++) {
+        int ch = chars[startOffset+i];
+        if (fst.findTargetArc(ch, arc, arc, i == 0, fstReader) == null) {
+          break; // continue to next position
+        }
+        output += arc.output.intValue();
+        if (arc.isFinal()) {
+          final int finalOutput = output + arc.nextFinalOutput.intValue();
+          result.add(finalOutput);
+        }
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/package-info.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/package-info.java
new file mode 100644
index 0000000..f3fb377
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/dict/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Korean dictionary implementation.
+ */
+package org.apache.lucene.analysis.ko.dict;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/package-info.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/package-info.java
new file mode 100644
index 0000000..200fd04
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Analyzer for Korean.
+ */
+package org.apache.lucene.analysis.ko;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttribute.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttribute.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttribute.java
new file mode 100644
index 0000000..1963c98
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttribute.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tokenattributes;
+
+import org.apache.lucene.analysis.ko.POS.Tag;
+import org.apache.lucene.analysis.ko.POS.Type;
+import org.apache.lucene.analysis.ko.Token;
+import org.apache.lucene.analysis.ko.dict.Dictionary.Morpheme;
+import org.apache.lucene.util.Attribute;
+
+/**
+ * Part of Speech attributes for Korean.
+ * @lucene.experimental
+ */
+public interface PartOfSpeechAttribute extends Attribute {
+  /**
+   * Get the {@link Type} of the token.
+   */
+  Type getPOSType();
+
+  /**
+   * Get the left part of speech of the token.
+   */
+  Tag getLeftPOS();
+
+  /**
+   * Get the right part of speech of the token.
+   */
+  Tag getRightPOS();
+
+  /**
+   * Get the {@link Morpheme} decomposition of the token.
+   */
+  Morpheme[] getMorphemes();
+
+  /**
+   * Set the current token.
+   */
+  void setToken(Token token);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttributeImpl.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttributeImpl.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttributeImpl.java
new file mode 100644
index 0000000..2e51689
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/PartOfSpeechAttributeImpl.java
@@ -0,0 +1,92 @@
+/*
+ * 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.tokenattributes;
+
+import org.apache.lucene.analysis.ko.POS.Type;
+import org.apache.lucene.analysis.ko.POS.Tag;
+import org.apache.lucene.analysis.ko.Token;
+import org.apache.lucene.analysis.ko.dict.Dictionary.Morpheme;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeReflector;
+
+/**
+ * Part of Speech attributes for Korean.
+ * @lucene.experimental
+ */
+public class PartOfSpeechAttributeImpl extends AttributeImpl implements PartOfSpeechAttribute, Cloneable {
+  private Token token;
+
+  @Override
+  public Type getPOSType() {
+    return token == null ? null : token.getPOSType();
+  }
+
+  @Override
+  public Tag getLeftPOS() {
+    return token == null ? null : token.getLeftPOS();
+  }
+
+  @Override
+  public Tag getRightPOS() {
+    return token == null ? null : token.getRightPOS();
+  }
+
+  @Override
+  public Morpheme[] getMorphemes() {
+    return token == null ? null : token.getMorphemes();
+  }
+
+  @Override
+  public void setToken(Token token) {
+    this.token = token;
+  }
+
+  @Override
+  public void clear() {
+    token = null;
+  }
+
+  @Override
+  public void reflectWith(AttributeReflector reflector) {
+    reflector.reflect(PartOfSpeechAttribute.class, "posType", getPOSType().name());
+    Tag leftPOS = getLeftPOS();
+    reflector.reflect(PartOfSpeechAttribute.class, "leftPOS", leftPOS.name() + "(" + leftPOS.description() + ")");
+    Tag rightPOS = getRightPOS();
+    reflector.reflect(PartOfSpeechAttribute.class, "rightPOS", rightPOS.name() + "(" + rightPOS.description() + ")");
+    reflector.reflect(PartOfSpeechAttribute.class, "morphemes", displayMorphemes(getMorphemes()));
+  }
+
+  private String displayMorphemes(Morpheme[] morphemes) {
+    if (morphemes == null) {
+      return null;
+    }
+    StringBuilder builder = new StringBuilder();
+    for (Morpheme morpheme : morphemes) {
+      if (builder.length() > 0) {
+        builder.append("+");
+      }
+      builder.append(morpheme.surfaceForm + "/" + morpheme.posTag.name() + "(" + morpheme.posTag.description() + ")");
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public void copyTo(AttributeImpl target) {
+    PartOfSpeechAttribute t = (PartOfSpeechAttribute) target;
+    t.setToken(token);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttribute.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttribute.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttribute.java
new file mode 100644
index 0000000..a5eb24f
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttribute.java
@@ -0,0 +1,38 @@
+/*
+ * 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.tokenattributes;
+
+import org.apache.lucene.analysis.ko.Token;
+import org.apache.lucene.util.Attribute;
+
+/**
+ * Attribute for Korean reading data
+ * <p>
+ * Note: in some cases this value may not be applicable, and will be null.
+ * @lucene.experimental
+ */
+public interface ReadingAttribute extends Attribute {
+  /**
+   * Get the reading of the token.
+   */
+  String getReading();
+
+  /**
+   * Set the current token.
+   */
+  void setToken(Token token);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttributeImpl.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttributeImpl.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttributeImpl.java
new file mode 100644
index 0000000..a64a4b3
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/ReadingAttributeImpl.java
@@ -0,0 +1,55 @@
+/*
+ * 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.tokenattributes;
+
+import org.apache.lucene.analysis.ko.Token;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeReflector;
+
+/**
+ * Attribute for Korean reading data
+ * @lucene.experimental
+ */
+public class ReadingAttributeImpl extends AttributeImpl implements ReadingAttribute, Cloneable {
+  private Token token;
+  
+  @Override
+  public String getReading() {
+    return token == null ? null : token.getReading();
+  }
+
+  @Override
+  public void setToken(Token token) {
+    this.token = token;
+  }
+
+  @Override
+  public void clear() {
+    token = null;
+  }
+
+  @Override
+  public void copyTo(AttributeImpl target) {
+    ReadingAttribute t = (ReadingAttribute) target;
+    t.setToken(token);
+  }
+  
+  @Override
+  public void reflectWith(AttributeReflector reflector) {
+    reflector.reflect(ReadingAttribute.class, "reading", getReading());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/package-info.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/package-info.java
new file mode 100644
index 0000000..5a33cfb
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/tokenattributes/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Additional Korean-specific Attributes for text analysis.
+ */
+package org.apache.lucene.analysis.ko.tokenattributes;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/CSVUtil.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/CSVUtil.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/CSVUtil.java
new file mode 100644
index 0000000..f911b55
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/CSVUtil.java
@@ -0,0 +1,95 @@
+/*
+ * 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.util.ArrayList;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Utility class for parsing CSV text
+ */
+public final class CSVUtil {
+  private static final char QUOTE = '"';
+  
+  private static final char COMMA = ',';
+  
+  private static final Pattern QUOTE_REPLACE_PATTERN = Pattern.compile("^\"([^\"]+)\"$");
+  
+  private static final String ESCAPED_QUOTE = "\"\"";
+  
+  private CSVUtil() {} // no instance!!!
+  
+  /**
+   * Parse CSV line
+   * @param line line containing csv-encoded data
+   * @return Array of values
+   */
+  public static String[] parse(String line) {
+    boolean insideQuote = false;
+    ArrayList<String> result = new ArrayList<>();
+    int quoteCount = 0;
+    StringBuilder sb = new StringBuilder();
+    for(int i = 0; i < line.length(); i++) {
+      char c = line.charAt(i);
+      
+      if(c == QUOTE) {
+        insideQuote = !insideQuote;
+        quoteCount++;
+      }
+      
+      if(c == COMMA && !insideQuote) {
+        String value = sb.toString();
+        value = unQuoteUnEscape(value);
+        result.add(value);
+        sb.setLength(0);
+        continue;
+      }
+      
+      sb.append(c);
+    }
+    
+    result.add(sb.toString());
+    
+    // Validate
+    if(quoteCount % 2 != 0) {
+      return new String[0];
+    }
+    
+    return result.toArray(new String[result.size()]);
+  }
+  
+  private static String unQuoteUnEscape(String original) {
+    String result = original;
+    
+    // Unquote
+    if (result.indexOf('\"') >= 0) {
+      Matcher m = QUOTE_REPLACE_PATTERN.matcher(original);
+      if(m.matches()) {
+        result = m.group(1);
+      }
+    
+      // Unescape
+      if (result.indexOf(ESCAPED_QUOTE) >= 0) {
+        result = result.replace(ESCAPED_QUOTE, "\"");
+      }
+    }
+    
+    return result;
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/package-info.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/package-info.java
new file mode 100644
index 0000000..8a25fdf
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/util/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nori utility classes.
+ */
+package org.apache.lucene.analysis.ko.util;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/overview.html
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/overview.html b/lucene/analysis/nori/src/java/overview.html
new file mode 100644
index 0000000..8ed9efd
--- /dev/null
+++ b/lucene/analysis/nori/src/java/overview.html
@@ -0,0 +1,34 @@
+<!--
+ 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.
+-->
+<html>
+  <head>
+    <title>
+      Apache Lucene Nori Analyzer
+    </title>
+  </head>
+  <body>
+    Nori is a morphological analyzer for Korean text.
+    <p>
+      This module provides support for Korean text analysis, including features
+      such as part-of-speech tagging, part-of-speech filtering, decompounding and
+      hanja to hangul conversion.
+    </p>
+    <p>
+      For an introduction to Lucene's analysis API, see the {@link org.apache.lucene.analysis} package documentation.
+    </p>
+  </body>
+</html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory b/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
new file mode 100644
index 0000000..07a41c7
--- /dev/null
+++ b/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenFilterFactory
@@ -0,0 +1,16 @@
+#  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.
+
+org.apache.lucene.analysis.ko.KoreanReadingFormFilterFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory b/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
new file mode 100644
index 0000000..e541b0e
--- /dev/null
+++ b/lucene/analysis/nori/src/resources/META-INF/services/org.apache.lucene.analysis.util.TokenizerFactory
@@ -0,0 +1,16 @@
+#  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.
+
+org.apache.lucene.analysis.ko.KoreanTokenizerFactory

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/CharacterDefinition.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/CharacterDefinition.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/CharacterDefinition.dat
new file mode 100644
index 0000000..97b23bb
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/CharacterDefinition.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/ConnectionCosts.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/ConnectionCosts.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/ConnectionCosts.dat
new file mode 100644
index 0000000..7fad91e
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/ConnectionCosts.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$buffer.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$buffer.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$buffer.dat
new file mode 100644
index 0000000..6958664
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$buffer.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$fst.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$fst.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$fst.dat
new file mode 100644
index 0000000..17b531f
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$fst.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$posDict.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$posDict.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$posDict.dat
new file mode 100644
index 0000000..c4db798
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$posDict.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$targetMap.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$targetMap.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$targetMap.dat
new file mode 100644
index 0000000..7c0823c
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/TokenInfoDictionary$targetMap.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$buffer.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$buffer.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$buffer.dat
new file mode 100644
index 0000000..c9a18ee
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$buffer.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$posDict.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$posDict.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$posDict.dat
new file mode 100644
index 0000000..f1ceebd
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$posDict.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$targetMap.dat
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$targetMap.dat b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$targetMap.dat
new file mode 100644
index 0000000..abcdf90
Binary files /dev/null and b/lucene/analysis/nori/src/resources/org/apache/lucene/analysis/ko/dict/UnknownDictionary$targetMap.dat differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/StringMockResourceLoader.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/StringMockResourceLoader.java b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/StringMockResourceLoader.java
new file mode 100644
index 0000000..cc1ee00
--- /dev/null
+++ b/lucene/analysis/nori/src/test/org/apache/lucene/analysis/ko/StringMockResourceLoader.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.lucene.analysis.util.ResourceLoader;
+
+/** Fake resource loader for tests: works if you want to fake reading a single file */
+class StringMockResourceLoader implements ResourceLoader {
+  String text;
+
+  public StringMockResourceLoader(String text) {
+    this.text = text;
+  }
+
+  @Override
+  public <T> Class<? extends T> findClass(String cname, Class<T> expectedType) {
+    try {
+      return Class.forName(cname).asSubclass(expectedType);
+    } catch (Exception e) {
+      throw new RuntimeException("Cannot load class: " + cname, e);
+    }
+  }
+
+  @Override
+  public <T> T newInstance(String cname, Class<T> expectedType) {
+    Class<? extends T> clazz = findClass(cname, expectedType);
+    try {
+      return clazz.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Cannot create instance: " + cname, e);
+    }
+  }
+
+  @Override
+  public InputStream openResource(String resource) throws IOException {
+    return new ByteArrayInputStream(text.getBytes(StandardCharsets.UTF_8));
+  }
+}


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

Posted by ji...@apache.org.
LUCENE-8231: Add a new analysis module (nori) similar to Kuromoji but to handle Korean

This change adds a korean analyzer in a new analysis module named nori. It is similar
to Kuromoji but uses the mecab-ko-dic dictionary to perform morphological analysis of Korean
text.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8651fbaa
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8651fbaa
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8651fbaa

Branch: refs/heads/branch_7x
Commit: 8651fbaa8bc65fdfec397ad4e7ebe2b5acf78d7b
Parents: 49098e5
Author: Jim Ferenczi <ji...@apache.org>
Authored: Fri Apr 13 11:26:42 2018 +0200
Committer: Jim Ferenczi <ji...@apache.org>
Committed: Fri Apr 13 14:17:13 2018 +0200

----------------------------------------------------------------------
 dev-tools/idea/.idea/modules.xml                |   1 +
 dev-tools/idea/lucene/analysis/nori/nori.iml    |  22 +
 .../maven/lucene/analysis/nori/pom.xml.template |  75 ++
 .../maven/lucene/analysis/pom.xml.template      |   1 +
 lucene/CHANGES.txt                              |   4 +
 lucene/NOTICE.txt                               |  12 +
 lucene/analysis/README.txt                      |   5 +
 lucene/analysis/build.xml                       |   7 +-
 lucene/analysis/nori/build.xml                  | 135 +++
 lucene/analysis/nori/ivy.xml                    |  33 +
 .../lucene/analysis/ko/DecompoundToken.java     |  69 ++
 .../lucene/analysis/ko/DictionaryToken.java     | 100 ++
 .../lucene/analysis/ko/GraphvizFormatter.java   | 180 ++++
 .../lucene/analysis/ko/KoreanAnalyzer.java      |  78 ++
 .../ko/KoreanPartOfSpeechStopFilter.java        |  85 ++
 .../ko/KoreanPartOfSpeechStopFilterFactory.java |  51 +
 .../analysis/ko/KoreanReadingFormFilter.java    |  51 +
 .../ko/KoreanReadingFormFilterFactory.java      |  42 +
 .../lucene/analysis/ko/KoreanTokenizer.java     | 957 +++++++++++++++++++
 .../analysis/ko/KoreanTokenizerFactory.java     |  89 ++
 .../java/org/apache/lucene/analysis/ko/POS.java | 304 ++++++
 .../org/apache/lucene/analysis/ko/Token.java    | 125 +++
 .../analysis/ko/dict/BinaryDictionary.java      | 239 +++++
 .../analysis/ko/dict/CharacterDefinition.java   | 136 +++
 .../analysis/ko/dict/ConnectionCosts.java       |  96 ++
 .../lucene/analysis/ko/dict/Dictionary.java     |  83 ++
 .../analysis/ko/dict/TokenInfoDictionary.java   |  77 ++
 .../lucene/analysis/ko/dict/TokenInfoFST.java   |  85 ++
 .../analysis/ko/dict/UnknownDictionary.java     |  61 ++
 .../lucene/analysis/ko/dict/UserDictionary.java | 235 +++++
 .../lucene/analysis/ko/dict/package-info.java   |  21 +
 .../apache/lucene/analysis/ko/package-info.java |  21 +
 .../tokenattributes/PartOfSpeechAttribute.java  |  54 ++
 .../PartOfSpeechAttributeImpl.java              |  92 ++
 .../ko/tokenattributes/ReadingAttribute.java    |  38 +
 .../tokenattributes/ReadingAttributeImpl.java   |  55 ++
 .../ko/tokenattributes/package-info.java        |  21 +
 .../apache/lucene/analysis/ko/util/CSVUtil.java |  95 ++
 .../lucene/analysis/ko/util/package-info.java   |  21 +
 lucene/analysis/nori/src/java/overview.html     |  34 +
 ...ache.lucene.analysis.util.TokenFilterFactory |  16 +
 ...apache.lucene.analysis.util.TokenizerFactory |  16 +
 .../analysis/ko/dict/CharacterDefinition.dat    | Bin 0 -> 65564 bytes
 .../lucene/analysis/ko/dict/ConnectionCosts.dat | Bin 0 -> 11178837 bytes
 .../ko/dict/TokenInfoDictionary$buffer.dat      | Bin 0 -> 7245625 bytes
 .../ko/dict/TokenInfoDictionary$fst.dat         | Bin 0 -> 5640925 bytes
 .../ko/dict/TokenInfoDictionary$posDict.dat     | Bin 0 -> 2712 bytes
 .../ko/dict/TokenInfoDictionary$targetMap.dat   | Bin 0 -> 811783 bytes
 .../ko/dict/UnknownDictionary$buffer.dat        | Bin 0 -> 101 bytes
 .../ko/dict/UnknownDictionary$posDict.dat       | Bin 0 -> 1823 bytes
 .../ko/dict/UnknownDictionary$targetMap.dat     | Bin 0 -> 36 bytes
 .../analysis/ko/StringMockResourceLoader.java   |  58 ++
 .../lucene/analysis/ko/TestKoreanAnalyzer.java  | 109 +++
 ...TestKoreanPartOfSpeechStopFilterFactory.java |  59 ++
 .../ko/TestKoreanReadingFormFilter.java         |  75 ++
 .../ko/TestKoreanReadingFormFilterFactory.java  |  51 +
 .../lucene/analysis/ko/TestKoreanTokenizer.java | 355 +++++++
 .../analysis/ko/TestKoreanTokenizerFactory.java | 113 +++
 .../ko/dict/TestTokenInfoDictionary.java        | 113 +++
 .../analysis/ko/dict/UserDictionaryTest.java    |  62 ++
 .../org/apache/lucene/analysis/ko/userdict.txt  |   5 +
 .../ko/util/BinaryDictionaryWriter.java         | 282 ++++++
 .../ko/util/CharacterDefinitionWriter.java      |  94 ++
 .../ko/util/ConnectionCostsBuilder.java         |  67 ++
 .../analysis/ko/util/ConnectionCostsWriter.java |  75 ++
 .../analysis/ko/util/DictionaryBuilder.java     |  67 ++
 .../ko/util/TokenInfoDictionaryBuilder.java     | 150 +++
 .../ko/util/TokenInfoDictionaryWriter.java      |  49 +
 .../ko/util/UnknownDictionaryBuilder.java       | 134 +++
 .../ko/util/UnknownDictionaryWriter.java        |  65 ++
 .../analysis/ko/dict/UnknownDictionaryTest.java |  74 ++
 lucene/ivy-versions.properties                  |   1 +
 lucene/module-build.xml                         |  22 +
 73 files changed, 6001 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/dev-tools/idea/.idea/modules.xml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/.idea/modules.xml b/dev-tools/idea/.idea/modules.xml
index 207a675..f77c34a 100644
--- a/dev-tools/idea/.idea/modules.xml
+++ b/dev-tools/idea/.idea/modules.xml
@@ -13,6 +13,7 @@
 
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/common/analysis-common.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/icu/icu.iml" />
+      <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/nori/nori.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/kuromoji/kuromoji.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/morfologik/morfologik.iml" />
       <module group="Lucene/Analysis" filepath="$PROJECT_DIR$/lucene/analysis/opennlp/opennlp.iml" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/dev-tools/idea/lucene/analysis/nori/nori.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/analysis/nori/nori.iml b/dev-tools/idea/lucene/analysis/nori/nori.iml
new file mode 100644
index 0000000..aa2d18e
--- /dev/null
+++ b/dev-tools/idea/lucene/analysis/nori/nori.iml
@@ -0,0 +1,22 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module type="JAVA_MODULE" version="4">
+  <component name="NewModuleRootManager" inherit-compiler-output="false">
+    <output url="file://$MODULE_DIR$/../../../idea-build/lucene/analysis/godori/classes/java" />
+    <output-test url="file://$MODULE_DIR$/../../../idea-build/lucene/analysis/godori/classes/test" />
+    <exclude-output />
+    <content url="file://$MODULE_DIR$">
+      <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
+      <sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
+      <sourceFolder url="file://$MODULE_DIR$/src/tools/java" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/tools/test" isTestSource="true" />
+    </content>
+    <orderEntry type="inheritedJdk" />
+    <orderEntry type="sourceFolder" forTests="false" />
+    <orderEntry type="library" name="ICU library" level="project" />
+    <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
+    <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
+    <orderEntry type="module" module-name="analysis-common" />
+    <orderEntry type="module" module-name="lucene-core" />
+  </component>
+</module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/dev-tools/maven/lucene/analysis/nori/pom.xml.template
----------------------------------------------------------------------
diff --git a/dev-tools/maven/lucene/analysis/nori/pom.xml.template b/dev-tools/maven/lucene/analysis/nori/pom.xml.template
new file mode 100644
index 0000000..ac37a08
--- /dev/null
+++ b/dev-tools/maven/lucene/analysis/nori/pom.xml.template
@@ -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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.lucene</groupId>
+    <artifactId>lucene-parent</artifactId>
+    <version>@version@</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <groupId>org.apache.lucene</groupId>
+  <artifactId>lucene-analyzers-nori</artifactId>
+  <packaging>jar</packaging>
+  <name>Lucene Nori Korean Morphological Analyzer</name>
+  <description>
+    Lucene Nori Korean Morphological Analyzer
+  </description>
+  <properties>
+    <module-directory>lucene/analysis/nori</module-directory>
+    <relative-top-level>../../../..</relative-top-level>
+    <module-path>${relative-top-level}/${module-directory}</module-path>
+  </properties>
+  <scm>
+    <connection>scm:git:${vc-anonymous-base-url}</connection>
+    <developerConnection>scm:git:${vc-dev-base-url}</developerConnection>
+    <url>${vc-browse-base-url};f=${module-directory}</url>
+  </scm>
+  <dependencies>
+    <dependency> 
+      <!-- lucene-test-framework dependency must be declared before lucene-core -->
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-test-framework</artifactId>
+      <scope>test</scope>
+    </dependency>
+@lucene-analyzers-nori.internal.dependencies@
+@lucene-analyzers-nori.external.dependencies@
+@lucene-analyzers-nori.internal.test.dependencies@
+@lucene-analyzers-nori.external.test.dependencies@
+  </dependencies>
+  <build>
+    <sourceDirectory>${module-path}/src/java</sourceDirectory>
+    <testSourceDirectory>${module-path}/src/test</testSourceDirectory>
+    <resources>
+      <resource>
+        <directory>${module-path}/src/resources</directory>
+      </resource>
+    </resources>
+    <testResources>
+      <testResource>
+        <directory>${project.build.testSourceDirectory}</directory>
+        <excludes>
+          <exclude>**/*.java</exclude>
+        </excludes>
+      </testResource>
+    </testResources>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/dev-tools/maven/lucene/analysis/pom.xml.template
----------------------------------------------------------------------
diff --git a/dev-tools/maven/lucene/analysis/pom.xml.template b/dev-tools/maven/lucene/analysis/pom.xml.template
index 466ad30..dada0d5 100644
--- a/dev-tools/maven/lucene/analysis/pom.xml.template
+++ b/dev-tools/maven/lucene/analysis/pom.xml.template
@@ -35,6 +35,7 @@
     <module>icu</module>
     <module>kuromoji</module>
     <module>morfologik</module>
+    <module>nori</module>
     <module>opennlp</module>
     <module>phonetic</module>
     <module>smartcn</module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 743a6e1..29ef036 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -50,6 +50,10 @@ New Features
   helps merge policies in the soft-delete case to correctly implement retention
   policies without triggering uncessary merges. (Simon Willnauer, Mike McCandless)
 
+* LUCENE-8231: A new analysis module (nori) similar to Kuromoji
+  but to handle Korean using mecab-ko-dic and morphological analysis.
+  (Robert Muir, Jim Ferenczi)
+
 Bug Fixes
 
 * LUCENE-8234: Fixed bug in how spatial relationship is computed for

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/NOTICE.txt
----------------------------------------------------------------------
diff --git a/lucene/NOTICE.txt b/lucene/NOTICE.txt
index e25c211..4970d20 100644
--- a/lucene/NOTICE.txt
+++ b/lucene/NOTICE.txt
@@ -190,3 +190,15 @@ grants independently of ICOT any specific warranty to the user in
 writing, such person, organization or entity, will also be exempted
 from and not be held liable to the user for any such damages as noted
 above as far as the program is concerned.
+
+===========================================================================
+Nori Korean Morphological Analyzer - Apache Lucene Integration
+===========================================================================
+
+This software includes a binary and/or source version of data from
+
+  mecab-ko-dic-2.0.3-20170922
+
+which can be obtained from
+
+  https://bitbucket.org/eunjeon/mecab-ko-dic/downloads/mecab-ko-dic-2.0.3-20170922.tar.gz

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/README.txt
----------------------------------------------------------------------
diff --git a/lucene/analysis/README.txt b/lucene/analysis/README.txt
index c68584e..8a9c8d9 100644
--- a/lucene/analysis/README.txt
+++ b/lucene/analysis/README.txt
@@ -28,6 +28,9 @@ lucene-analyzers-kuromoji-XX.jar
 lucene-analyzers-morfologik-XX.jar
   An analyzer using the Morfologik stemming library.
 
+lucene-analyzers-nori-XX.jar
+  An analyzer with morphological analysis for Korean.
+
 lucene-analyzers-opennlp-XX.jar
   An analyzer using the OpenNLP natural-language processing library.
 
@@ -52,6 +55,7 @@ common/src/java
 icu/src/java
 kuromoji/src/java
 morfologik/src/java
+nori/src/java
 opennlp/src/java
 phonetic/src/java
 smartcn/src/java
@@ -63,6 +67,7 @@ common/src/test
 icu/src/test
 kuromoji/src/test
 morfologik/src/test
+nori/src/test
 opennlp/src/test
 phonetic/src/test
 smartcn/src/test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/build.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/build.xml b/lucene/analysis/build.xml
index ed1566c..95dd7dc 100644
--- a/lucene/analysis/build.xml
+++ b/lucene/analysis/build.xml
@@ -25,6 +25,7 @@
       - icu: Analyzers that use functionality from ICU
       - kuromoji: Japanese Morphological Analyzer
       - morfologik: Morfologik Stemmer
+      - nori: Korean Morphological Analyzer
       - smartcn: Smart Analyzer for Simplified Chinese Text
       - stempel: Algorithmic Stemmer for Polish
       - uima: UIMA Analysis module
@@ -65,6 +66,10 @@
     <ant dir="morfologik" />
   </target>
 
+  <target name="nori">
+    <ant dir="nori" />
+  </target>
+
   <target name="opennlp">
     <ant dir="opennlp" />
   </target>
@@ -86,7 +91,7 @@
   </target>
 
   <target name="default" depends="compile"/>
-  <target name="compile" depends="common,icu,kuromoji,morfologik,opennlp,phonetic,smartcn,stempel,uima" />
+  <target name="compile" depends="common,icu,kuromoji,morfologik,nori,opennlp,phonetic,smartcn,stempel,uima" />
 
   <target name="clean">
     <forall-analyzers target="clean"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/build.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/build.xml b/lucene/analysis/nori/build.xml
new file mode 100644
index 0000000..dacf3a9
--- /dev/null
+++ b/lucene/analysis/nori/build.xml
@@ -0,0 +1,135 @@
+<?xml version="1.0"?>
+
+<!--
+    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.
+ -->
+
+<project name="analyzers-nori" default="default" xmlns:ivy="antlib:org.apache.ivy.ant">
+
+  <description>
+    Korean Morphological Analyzer
+  </description>
+
+  <!-- currently whether rat detects this as binary or not
+       is platform dependent?! -->
+  <property name="rat.excludes" value="**/*.txt,**/bocchan.utf-8"/>
+  <property name="rat.additional-includes" value="src/tools/**"/>
+
+  <!-- we don't want to pull in ipadic/naist etc -->
+  <property name="ivy.default.configuration" value="default"/>
+  <import file="../analysis-module-build.xml"/>
+
+  <!-- default configuration for Korean: uses mecab-ko-dic -->
+  <property name="dict.type" value="mecab-ko-dic"/>
+  <property name="dict.version" value="mecab-ko-dic-2.0.3-20170922" />
+
+  <property name="dict.src.file" value="${dict.version}.tar.gz" />
+  <property name="dict.src.dir" value="${build.dir}/${dict.version}" />
+  <property name="dict.encoding" value="utf-8"/>
+  <property name="dict.normalize" value="false"/>
+  <property name="dict.target.dir" location="${resources.dir}"/>
+
+  <available type="dir" file="${build.dir}/${dict.version}" property="mecab-ko.dict.available"/>
+
+  <path id="classpath">
+    <pathelement path="${analyzers-common.jar}"/>
+    <path refid="base.classpath"/>
+  </path>
+
+  <target name="compile-core" depends="jar-analyzers-common, common.compile-core" />
+  <target name="download-dict" depends="ivy-availability-check,ivy-fail,ivy-configure" unless="mecab-ko.dict.available">
+    <ivy:retrieve pattern="${build.dir}/${dict.src.file}" conf="${dict.type}" symlink="${ivy.symlink}"/>
+    <!-- TODO: we should checksum too -->
+    <gunzip src="${build.dir}/${dict.src.file}"/>
+    <untar src="${build.dir}/${dict.version}.tar" dest="${build.dir}"/>
+  </target>
+
+  <path id="tools.dependencies">
+    <fileset dir="../icu/lib"/>
+  </path>
+
+  <path id="tools.classpath">
+    <path refid="classpath"/>
+    <path refid="tools.dependencies"/>
+    <pathelement location="${build.dir}/classes/java"/>
+    <pathelement location="${build.dir}/classes/tools"/>
+  </path>
+
+  <path id="tools.test.classpath">
+    <path refid="tools.classpath"/>
+    <path refid="test.base.classpath"/>
+    <pathelement location="${build.dir}/classes/tools-test"/>
+  </path>
+
+  <target name="build-dict" depends="compile-tools, download-dict">
+    <sequential>
+      <delete verbose="true">
+        <fileset dir="${resources.dir}/org/apache/lucene/analysis/ko/dict" includes="**/*"/>
+      </delete>
+      <!-- TODO: optimize the dictionary construction a bit so that you don't need 1G -->
+      <java fork="true" failonerror="true" maxmemory="1g" classname="org.apache.lucene.analysis.ko.util.DictionaryBuilder">
+        <classpath>
+          <path refid="tools.classpath"/>
+        </classpath>
+        <assertions>
+          <enable package="org.apache.lucene"/>
+        </assertions>
+        <arg value="${dict.src.dir}"/>
+        <arg value="${dict.target.dir}"/>
+        <arg value="${dict.encoding}"/>
+        <arg value="${dict.normalize}"/>
+      </java>
+    </sequential>
+  </target>
+
+  <!-- we don't actually need to compile this thing, we just want its lib -->
+  <target name="resolve-icu">
+    <ant dir="../icu/" target="resolve" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+    </ant>
+  </target>
+
+  <target name="compile-tools" depends="resolve-icu, compile-core, common.compile-tools">
+    <compile
+        srcdir="src/tools/java"
+        destdir="${build.dir}/classes/tools">
+      <classpath>
+        <path refid="tools.classpath"/>
+      </classpath>
+    </compile>
+  </target>
+
+  <target name="compile-tools-tests" depends="compile-tools">
+    <compile
+        srcdir="src/tools/test"
+        destdir="${build.dir}/classes/tools-test">
+      <classpath>
+        <path refid="tools.test.classpath"/>
+        <pathelement path="src/tools/test"/>
+      </classpath>
+    </compile>
+  </target>
+
+  <target name="test-tools" depends="compile-tools-tests">
+    <test-macro dataDir="src/tools/test" junit.classpath="tools.test.classpath"/>
+  </target>
+
+  <target name="compile-test" depends="module-build.compile-test, compile-tools-tests"/>
+  <!-- TODO: not until we properly make 'test-tools' work with clover etc
+  <target name="test" depends="module-build.test, test-tools"/> -->
+
+  <target name="regenerate" depends="build-dict"/>
+</project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/ivy.xml
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/ivy.xml b/lucene/analysis/nori/ivy.xml
new file mode 100644
index 0000000..8d32937
--- /dev/null
+++ b/lucene/analysis/nori/ivy.xml
@@ -0,0 +1,33 @@
+<!--
+   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.
+-->
+<ivy-module version="2.0">
+  <info organisation="org.apache.lucene" module="analyzers-nori"/>
+
+  <configurations defaultconfmapping="mecab-ko-dic->default"> <!-- 'master' conf not available to map to -->
+    <conf name="default" description="explicitly declare this configuration in order to not download dictionaries unless explicitly called for"/>
+    <conf name="mecab-ko-dic" description="mecab-ko dictionary for Korean" transitive="false"/>
+  </configurations>
+
+  <dependencies>
+    <dependency org="mecab" name="mecab-ko-dic" rev="${/mecab/mecab-ko-dic}" conf="mecab-ko-dic">
+      <artifact name="mecab-ko-dic" type=".tar.gz" url="https://bitbucket.org/eunjeon/mecab-ko-dic/downloads/mecab-ko-dic-2.0.3-20170922.tar.gz" />
+    </dependency>
+    <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/>
+  </dependencies>
+</ivy-module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DecompoundToken.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DecompoundToken.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DecompoundToken.java
new file mode 100644
index 0000000..a44a2d8
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DecompoundToken.java
@@ -0,0 +1,69 @@
+/*
+ * 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 org.apache.lucene.analysis.ko.dict.Dictionary;
+
+/**
+ * A token that was generated from a compound.
+ */
+public class DecompoundToken extends Token {
+  private final POS.Tag posTag;
+
+  /**
+   *  Creates a new DecompoundToken
+   * @param posTag The part of speech of the token.
+   * @param surfaceForm The surface form of the token.
+   * @param startOffset The start offset of the token in the analyzed text.
+   * @param endOffset The end offset of the token in the analyzed text.
+   */
+  public DecompoundToken(POS.Tag posTag, String surfaceForm, int startOffset, int endOffset) {
+    super(surfaceForm.toCharArray(), 0, surfaceForm.length(), startOffset, endOffset);
+    this.posTag = posTag;
+  }
+
+  @Override
+  public String toString() {
+    return "DecompoundToken(\"" + getSurfaceFormString() + "\" pos=" + getStartOffset() + " length=" + getLength() +
+        " startOffset=" + getStartOffset() + " endOffset=" + getEndOffset() + ")";
+  }
+
+  @Override
+  public POS.Type getPOSType() {
+    return POS.Type.MORPHEME;
+  }
+
+  @Override
+  public POS.Tag getLeftPOS() {
+    return posTag;
+  }
+
+  @Override
+  public POS.Tag getRightPOS() {
+    return posTag;
+  }
+
+  @Override
+  public String getReading() {
+    return null;
+  }
+
+  @Override
+  public Dictionary.Morpheme[] getMorphemes() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DictionaryToken.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DictionaryToken.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DictionaryToken.java
new file mode 100644
index 0000000..3efb119
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/DictionaryToken.java
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.lucene.analysis.ko.dict.Dictionary;
+
+/**
+ * A token stored in a {@link Dictionary}.
+ */
+public class DictionaryToken extends Token {
+  private final int wordId;
+  private final KoreanTokenizer.Type type;
+  private final Dictionary dictionary;
+
+  public DictionaryToken(KoreanTokenizer.Type type, Dictionary dictionary, int wordId, char[] surfaceForm,
+                         int offset, int length, int startOffset, int endOffset) {
+    super(surfaceForm, offset, length, startOffset, endOffset);
+    this.type = type;
+    this.dictionary = dictionary;
+    this.wordId = wordId;
+  }
+
+  @Override
+  public String toString() {
+    return "DictionaryToken(\"" + getSurfaceFormString() + "\" pos=" + getStartOffset() + " length=" + getLength() +
+        " posLen=" + getPositionLength() + " type=" + type + " wordId=" + wordId +
+        " leftID=" + dictionary.getLeftId(wordId) + ")";
+  }
+
+  /**
+   * Returns the type of this token
+   * @return token type, not null
+   */
+  public KoreanTokenizer.Type getType() {
+    return type;
+  }
+
+  /**
+   * Returns true if this token is known word
+   * @return true if this token is in standard dictionary. false if not.
+   */
+  public boolean isKnown() {
+    return type == KoreanTokenizer.Type.KNOWN;
+  }
+
+  /**
+   * Returns true if this token is unknown word
+   * @return true if this token is unknown word. false if not.
+   */
+  public boolean isUnknown() {
+    return type == KoreanTokenizer.Type.UNKNOWN;
+  }
+
+  /**
+   * Returns true if this token is defined in user dictionary
+   * @return true if this token is in user dictionary. false if not.
+   */
+  public boolean isUser() {
+    return type == KoreanTokenizer.Type.USER;
+  }
+
+  @Override
+  public POS.Type getPOSType() {
+    return dictionary.getPOSType(wordId);
+  }
+
+  @Override
+  public POS.Tag getLeftPOS() {
+    return dictionary.getLeftPOS(wordId);
+  }
+
+  @Override
+  public POS.Tag getRightPOS() {
+    return dictionary.getRightPOS(wordId);
+  }
+
+  @Override
+  public String getReading() {
+    return dictionary.getReading(wordId);
+  }
+
+  @Override
+  public Dictionary.Morpheme[] getMorphemes() {
+    return dictionary.getMorphemes(wordId, getSurfaceForm(), getOffset(), getLength());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/GraphvizFormatter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/GraphvizFormatter.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/GraphvizFormatter.java
new file mode 100644
index 0000000..9feb354
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/GraphvizFormatter.java
@@ -0,0 +1,180 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.ko.KoreanTokenizer.Position;
+import org.apache.lucene.analysis.ko.KoreanTokenizer.WrappedPositionArray;
+import org.apache.lucene.analysis.ko.dict.ConnectionCosts;
+import org.apache.lucene.analysis.ko.dict.Dictionary;
+
+
+// TODO: would be nice to show 2nd best path in a diff't
+// color...
+
+/**
+ * Outputs the dot (graphviz) string for the viterbi lattice.
+ */
+public class GraphvizFormatter {
+  
+  private final static String BOS_LABEL = "BOS";
+  
+  private final static String EOS_LABEL = "EOS";
+  
+  private final static String FONT_NAME = "Helvetica";
+  
+  private final ConnectionCosts costs;
+  
+  private final Map<String, String> bestPathMap;
+  
+  private final StringBuilder sb = new StringBuilder();
+  
+  public GraphvizFormatter(ConnectionCosts costs) {
+    this.costs = costs;
+    this.bestPathMap = new HashMap<>();
+    sb.append(formatHeader());
+    sb.append("  init [style=invis]\n");
+    sb.append("  init -> 0.0 [label=\"" + BOS_LABEL + "\"]\n");
+  }
+
+  public String finish() {
+    sb.append(formatTrailer());
+    return sb.toString();
+  }
+
+  // Backtraces another incremental fragment:
+  void onBacktrace(KoreanTokenizer tok, WrappedPositionArray positions, int lastBackTracePos, Position endPosData, int fromIDX, char[] fragment, boolean isEnd) {
+    setBestPathMap(positions, lastBackTracePos, endPosData, fromIDX);
+    sb.append(formatNodes(tok, positions, lastBackTracePos, endPosData, fragment));
+    if (isEnd) {
+      sb.append("  fini [style=invis]\n");
+      sb.append("  ");
+      sb.append(getNodeID(endPosData.pos, fromIDX));
+      sb.append(" -> fini [label=\"" + EOS_LABEL + "\"]");
+    }
+  }
+
+  // Records which arcs make up the best bath:
+  private void setBestPathMap(WrappedPositionArray positions, int startPos, Position endPosData, int fromIDX) {
+    bestPathMap.clear();
+
+    int pos = endPosData.pos;
+    int bestIDX = fromIDX;
+    while (pos > startPos) {
+      final Position posData = positions.get(pos);
+
+      final int backPos = posData.backPos[bestIDX];
+      final int backIDX = posData.backIndex[bestIDX];
+
+      final String toNodeID = getNodeID(pos, bestIDX);
+      final String fromNodeID = getNodeID(backPos, backIDX);
+      
+      assert !bestPathMap.containsKey(fromNodeID);
+      assert !bestPathMap.containsValue(toNodeID);
+      bestPathMap.put(fromNodeID, toNodeID);
+      pos = backPos;
+      bestIDX = backIDX;
+    }
+  }
+  
+  private String formatNodes(KoreanTokenizer tok, WrappedPositionArray positions, int startPos, Position endPosData, char[] fragment) {
+
+    StringBuilder sb = new StringBuilder();
+    // Output nodes
+    for (int pos = startPos+1; pos <= endPosData.pos; pos++) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        sb.append("  ");
+        sb.append(getNodeID(pos, idx));
+        sb.append(" [label=\"");
+        sb.append(pos);
+        sb.append(": ");
+        sb.append(posData.lastRightID[idx]);
+        sb.append("\"]\n");
+      }
+    }
+
+    // Output arcs
+    for (int pos = endPosData.pos; pos > startPos; pos--) {
+      final Position posData = positions.get(pos);
+      for(int idx=0;idx<posData.count;idx++) {
+        final Position backPosData = positions.get(posData.backPos[idx]);
+        final String toNodeID = getNodeID(pos, idx);
+        final String fromNodeID = getNodeID(posData.backPos[idx], posData.backIndex[idx]);
+
+        sb.append("  ");
+        sb.append(fromNodeID);
+        sb.append(" -> ");
+        sb.append(toNodeID);
+
+        final String attrs;
+        if (toNodeID.equals(bestPathMap.get(fromNodeID))) {
+          // This arc is on best path
+          attrs = " color=\"#40e050\" fontcolor=\"#40a050\" penwidth=3 fontsize=20";
+        } else {
+          attrs = "";
+        }
+
+        final Dictionary dict = tok.getDict(posData.backType[idx]);
+        final int wordCost = dict.getWordCost(posData.backID[idx]);
+        final int bgCost = costs.get(backPosData.lastRightID[posData.backIndex[idx]],
+                                     dict.getLeftId(posData.backID[idx]));
+
+        final String surfaceForm = new String(fragment,
+                                              posData.backPos[idx] - startPos,
+                                              pos - posData.backPos[idx]);
+        
+        sb.append(" [label=\"");
+        sb.append(surfaceForm);
+        sb.append(' ');
+        sb.append(wordCost);
+        if (bgCost >= 0) {
+          sb.append('+');
+        }
+        sb.append(bgCost);
+        sb.append("\"");
+        sb.append(attrs);
+        sb.append("]\n");
+      }
+    }
+    return sb.toString();
+  }
+  
+  private String formatHeader() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("digraph viterbi {\n");
+    sb.append("  graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\"];\n");
+    //sb.append("  // A2 paper size\n");
+    //sb.append("  size = \"34.4,16.5\";\n");
+    //sb.append("  // try to fill paper\n");
+    //sb.append("  ratio = fill;\n");
+    sb.append("  edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]\n");
+    sb.append("  node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]\n");
+    
+    return sb.toString();
+  }
+  
+  private String formatTrailer() {
+    return "}";
+  }
+  
+  private String getNodeID(int pos, int idx) {
+    return pos + "." + idx;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
new file mode 100644
index 0000000..76023bb
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanAnalyzer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.LowerCaseFilter;
+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 static org.apache.lucene.analysis.TokenStream.DEFAULT_TOKEN_ATTRIBUTE_FACTORY;
+
+/**
+ * Analyzer for Korean that uses morphological analysis.
+ * @see KoreanTokenizer
+ * @lucene.experimental
+ */
+public class KoreanAnalyzer extends Analyzer {
+  private final UserDictionary userDict;
+  private final KoreanTokenizer.DecompoundMode mode;
+  private final Set<POS.Tag> stopTags;
+  private final boolean outputUnknownUnigrams;
+
+  /**
+   * Creates a new KoreanAnalyzer.
+   */
+  public KoreanAnalyzer() {
+    this(null, KoreanTokenizer.DEFAULT_DECOMPOUND, KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS, false);
+  }
+
+  /**
+   * Creates a new KoreanAnalyzer.
+   *
+   * @param userDict Optional: if non-null, user dictionary.
+   * @param mode Decompound mode.
+   * @param stopTags The set of part of speech that should be filtered.
+   * @param outputUnknownUnigrams If true outputs unigrams for unknown words.
+   */
+  public KoreanAnalyzer(UserDictionary userDict, DecompoundMode mode, Set<POS.Tag> stopTags, boolean outputUnknownUnigrams) {
+    super();
+    this.userDict = userDict;
+    this.mode = mode;
+    this.stopTags = stopTags;
+    this.outputUnknownUnigrams = outputUnknownUnigrams;
+  }
+  
+  @Override
+  protected TokenStreamComponents createComponents(String fieldName) {
+    Tokenizer tokenizer = new KoreanTokenizer(DEFAULT_TOKEN_ATTRIBUTE_FACTORY, userDict, mode, outputUnknownUnigrams);
+    TokenStream stream = new KoreanPartOfSpeechStopFilter(tokenizer, stopTags);
+    stream = new KoreanReadingFormFilter(stream);
+    stream = new LowerCaseFilter(stream);
+    return new TokenStreamComponents(tokenizer, stream);
+  }
+
+  @Override
+  protected TokenStream normalize(String fieldName, TokenStream in) {
+    TokenStream result = new LowerCaseFilter(in);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilter.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilter.java
new file mode 100644
index 0000000..4fa7524
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilter.java
@@ -0,0 +1,85 @@
+/*
+ * 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.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.analysis.FilteringTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.ko.tokenattributes.PartOfSpeechAttribute;
+
+/**
+ * Removes tokens that match a set of part-of-speech tags.
+ * @lucene.experimental
+ */
+public final class KoreanPartOfSpeechStopFilter extends FilteringTokenFilter {
+  private final Set<POS.Tag> stopTags;
+  private final PartOfSpeechAttribute posAtt = addAttribute(PartOfSpeechAttribute.class);
+
+  /**
+   * Default list of tags to filter.
+   */
+  public static final Set<POS.Tag> DEFAULT_STOP_TAGS = Arrays.asList(
+      POS.Tag.E,
+      POS.Tag.IC,
+      POS.Tag.J,
+      POS.Tag.MAG,
+      POS.Tag.MAJ,
+      POS.Tag.MM,
+      POS.Tag.SP,
+      POS.Tag.SSC,
+      POS.Tag.SSO,
+      POS.Tag.SC,
+      POS.Tag.SE,
+      POS.Tag.XPN,
+      POS.Tag.XSA,
+      POS.Tag.XSN,
+      POS.Tag.XSV,
+      POS.Tag.UNA,
+      POS.Tag.NA,
+      POS.Tag.VSV
+  ).stream().collect(Collectors.toSet());
+
+  /**
+   * Create a new {@link KoreanPartOfSpeechStopFilter} with the default
+   * list of stop tags {@link #DEFAULT_STOP_TAGS}.
+   *
+   * @param input    the {@link TokenStream} to consume
+   */
+  public KoreanPartOfSpeechStopFilter(TokenStream input) {
+    this(input, DEFAULT_STOP_TAGS);
+  }
+
+  /**
+   * Create a new {@link KoreanPartOfSpeechStopFilter}.
+   * @param input    the {@link TokenStream} to consume
+   * @param stopTags the part-of-speech tags that should be removed
+   */
+  public KoreanPartOfSpeechStopFilter(TokenStream input, Set<POS.Tag> stopTags) {
+    super(input);
+    this.stopTags = stopTags;
+  }
+
+  @Override
+  protected boolean accept() {
+    final POS.Tag leftPOS = posAtt.getLeftPOS();
+    return leftPOS == null || !stopTags.contains(leftPOS);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.java
new file mode 100644
index 0000000..010abc8
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanPartOfSpeechStopFilterFactory.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.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link KoreanPartOfSpeechStopFilter}.
+ * @lucene.experimental
+ */
+public class KoreanPartOfSpeechStopFilterFactory extends TokenFilterFactory {
+  private Set<POS.Tag> stopTags;
+
+  /** Creates a new JapanesePartOfSpeechStopFilterFactory */
+  public KoreanPartOfSpeechStopFilterFactory(Map<String,String> args) {
+    super(args);
+    Set<String> stopTagStr = getSet(args, "tags");
+    if (stopTagStr == null) {
+      stopTags = KoreanPartOfSpeechStopFilter.DEFAULT_STOP_TAGS;
+    } else {
+      stopTags = stopTagStr.stream().map(POS::resolveTag).collect(Collectors.toSet());
+    }
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+  
+  @Override
+  public TokenStream create(TokenStream stream) {
+      return new KoreanPartOfSpeechStopFilter(stream, stopTags);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilter.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilter.java
new file mode 100644
index 0000000..8b7e6cb
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilter.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 org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.ko.tokenattributes.ReadingAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+/**
+ * Replaces term text with the {@link ReadingAttribute} which is
+ * the Hangul transcription of Hanja characters.
+ * @lucene.experimental
+ */
+public final class KoreanReadingFormFilter extends TokenFilter {
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final ReadingAttribute readingAtt = addAttribute(ReadingAttribute.class);
+
+  public KoreanReadingFormFilter(TokenStream input) {
+    super(input);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (input.incrementToken()) {
+      String reading = readingAtt.getReading();
+      if (reading != null) {
+        termAtt.setEmpty().append(reading);
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
new file mode 100644
index 0000000..860a139
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanReadingFormFilterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+
+/**
+ * Factory for {@link KoreanReadingFormFilter}.
+ * @lucene.experimental
+ */
+public class KoreanReadingFormFilterFactory extends TokenFilterFactory {
+
+  /** Creates a new KoreanReadingFilterFactory */
+  public KoreanReadingFormFilterFactory(Map<String,String> args) {
+    super(args);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+  
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new KoreanReadingFormFilter(input);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizer.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizer.java b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizer.java
new file mode 100644
index 0000000..822853b
--- /dev/null
+++ b/lucene/analysis/nori/src/java/org/apache/lucene/analysis/ko/KoreanTokenizer.java
@@ -0,0 +1,957 @@
+/*
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.List;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
+import org.apache.lucene.analysis.ko.dict.ConnectionCosts;
+import org.apache.lucene.analysis.ko.dict.Dictionary;
+import org.apache.lucene.analysis.ko.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.ko.dict.TokenInfoFST;
+import org.apache.lucene.analysis.ko.dict.UnknownDictionary;
+import org.apache.lucene.analysis.ko.dict.UserDictionary;
+import org.apache.lucene.analysis.ko.tokenattributes.PartOfSpeechAttribute;
+import org.apache.lucene.analysis.ko.tokenattributes.ReadingAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.util.RollingCharBuffer;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.AttributeFactory;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.fst.FST;
+
+/**
+ * Tokenizer for Korean that uses morphological analysis.
+ * <p>
+ * This tokenizer sets a number of additional attributes:
+ * <ul>
+ *   <li>{@link PartOfSpeechAttribute} containing part-of-speech.
+ *   <li>{@link ReadingAttribute} containing reading.
+ * </ul>
+ * <p>
+ * This tokenizer uses a rolling Viterbi search to find the
+ * least cost segmentation (path) of the incoming characters.
+ * @lucene.experimental
+ */
+public final class KoreanTokenizer extends Tokenizer {
+
+  /**
+   * Token type reflecting the original source of this token
+   */
+  public enum Type {
+    /**
+     * Known words from the system dictionary.
+     */
+    KNOWN,
+    /**
+     * Unknown words (heuristically segmented).
+     */
+    UNKNOWN,
+    /**
+     * Known words from the user dictionary.
+     */
+    USER
+  }
+
+  /**
+   * Decompound mode: this determines how the tokenizer handles
+   * {@link POS.Type#COMPOUND}, {@link POS.Type#INFLECT} and {@link POS.Type#PREANALYSIS} tokens.
+   */
+  public enum DecompoundMode {
+    /**
+     * No decomposition for compound.
+     */
+    NONE,
+
+    /**
+     * Decompose compounds and discards the original form (default).
+     */
+    DISCARD,
+
+    /**
+     * Decompose compounds and keeps the original form.
+     */
+    MIXED
+  }
+
+  /**
+   * Default mode for the decompound of tokens ({@link DecompoundMode#DISCARD}.
+   */
+  public static final DecompoundMode DEFAULT_DECOMPOUND = DecompoundMode.DISCARD;
+
+  private static final boolean VERBOSE = false;
+
+  // For safety:
+  private static final int MAX_UNKNOWN_WORD_LENGTH = 1024;
+  private static final int MAX_BACKTRACE_GAP = 1024;
+
+  private final EnumMap<Type, Dictionary> dictionaryMap = new EnumMap<>(Type.class);
+
+  private final TokenInfoFST fst;
+  private final TokenInfoDictionary dictionary;
+  private final UnknownDictionary unkDictionary;
+  private final ConnectionCosts costs;
+  private final UserDictionary userDictionary;
+  private final CharacterDefinition characterDefinition;
+
+  private final FST.Arc<Long> arc = new FST.Arc<>();
+  private final FST.BytesReader fstReader;
+  private final IntsRef wordIdRef = new IntsRef();
+
+  private final FST.BytesReader userFSTReader;
+  private final TokenInfoFST userFST;
+
+  private final DecompoundMode mode;
+  private final boolean outputUnknownUnigrams;
+
+  private final RollingCharBuffer buffer = new RollingCharBuffer();
+
+  private final WrappedPositionArray positions = new WrappedPositionArray();
+
+  // True once we've hit the EOF from the input reader:
+  private boolean end;
+
+  // Last absolute position we backtraced from:
+  private int lastBackTracePos;
+
+  // Next absolute position to process:
+  private int pos;
+
+  // Already parsed, but not yet passed to caller, tokens:
+  private final List<Token> pending = new ArrayList<>();
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+  private final PartOfSpeechAttribute posAtt = addAttribute(PartOfSpeechAttribute.class);
+  private final ReadingAttribute readingAtt = addAttribute(ReadingAttribute.class);
+
+
+  /**
+   * Creates a new KoreanTokenizer with default parameters.
+   * <p>
+   * Uses the default AttributeFactory.
+   */
+  public KoreanTokenizer() {
+    this(DEFAULT_TOKEN_ATTRIBUTE_FACTORY, null, DEFAULT_DECOMPOUND, false);
+  }
+
+  /**
+   * Create a new KoreanTokenizer.
+   *
+   * @param factory the AttributeFactory to use
+   * @param userDictionary Optional: if non-null, user dictionary.
+   * @param mode Decompound mode.
+   * @param outputUnknownUnigrams If true outputs unigrams for unknown words.
+   */
+  public KoreanTokenizer(AttributeFactory factory, UserDictionary userDictionary, DecompoundMode mode, boolean outputUnknownUnigrams) {
+    super(factory);
+    this.mode = mode;
+    this.outputUnknownUnigrams = outputUnknownUnigrams;
+    dictionary = TokenInfoDictionary.getInstance();
+    fst = dictionary.getFST();
+    unkDictionary = UnknownDictionary.getInstance();
+    characterDefinition = unkDictionary.getCharacterDefinition();
+    this.userDictionary = userDictionary;
+    costs = ConnectionCosts.getInstance();
+    fstReader = fst.getBytesReader();
+    if (userDictionary != null) {
+      userFST = userDictionary.getFST();
+      userFSTReader = userFST.getBytesReader();
+    } else {
+      userFST = null;
+      userFSTReader = null;
+    }
+
+    buffer.reset(this.input);
+
+    resetState();
+
+    dictionaryMap.put(Type.KNOWN, dictionary);
+    dictionaryMap.put(Type.UNKNOWN, unkDictionary);
+    dictionaryMap.put(Type.USER, userDictionary);
+  }
+
+  private GraphvizFormatter dotOut;
+
+  /** Expert: set this to produce graphviz (dot) output of
+   *  the Viterbi lattice */
+  public void setGraphvizFormatter(GraphvizFormatter dotOut) {
+    this.dotOut = dotOut;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    buffer.reset(input);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    buffer.reset(input);
+    resetState();
+  }
+
+  private void resetState() {
+    positions.reset();
+    pos = 0;
+    end = false;
+    lastBackTracePos = 0;
+    pending.clear();
+
+    // Add BOS:
+    positions.get(0).add(0, 0, -1, -1, -1, -1, Type.KNOWN);
+  }
+
+  @Override
+  public void end() throws IOException {
+    super.end();
+    // Set final offset
+    int finalOffset = correctOffset(pos);
+    offsetAtt.setOffset(finalOffset, finalOffset);
+  }
+
+  // Holds all back pointers arriving to this position:
+  final static class Position {
+
+    int pos;
+
+    int count;
+
+    // maybe single int array * 5?
+    int[] costs = new int[8];
+    int[] lastRightID = new int[8];
+    int[] backPos = new int[8];
+    int[] backWordPos = new int[8];
+    int[] backIndex = new int[8];
+    int[] backID = new int[8];
+    Type[] backType = new Type[8];
+
+    public void grow() {
+      costs = ArrayUtil.grow(costs, 1+count);
+      lastRightID = ArrayUtil.grow(lastRightID, 1+count);
+      backPos = ArrayUtil.grow(backPos, 1+count);
+      backWordPos = ArrayUtil.grow(backWordPos, 1+count);
+      backIndex = ArrayUtil.grow(backIndex, 1+count);
+      backID = ArrayUtil.grow(backID, 1+count);
+
+      // NOTE: sneaky: grow separately because
+      // ArrayUtil.grow will otherwise pick a different
+      // length than the int[]s we just grew:
+      final Type[] newBackType = new Type[backID.length];
+      System.arraycopy(backType, 0, newBackType, 0, backType.length);
+      backType = newBackType;
+    }
+
+    public void add(int cost, int lastRightID, int backPos, int backRPos, int backIndex, int backID, Type backType) {
+      // NOTE: this isn't quite a true Viterbi search,
+      // because we should check if lastRightID is
+      // already present here, and only update if the new
+      // cost is less than the current cost, instead of
+      // simply appending.  However, that will likely hurt
+      // performance (usually we add a lastRightID only once),
+      // and it means we actually create the full graph
+      // intersection instead of a "normal" Viterbi lattice:
+      if (count == costs.length) {
+        grow();
+      }
+      this.costs[count] = cost;
+      this.lastRightID[count] = lastRightID;
+      this.backPos[count] = backPos;
+      this.backWordPos[count] = backRPos;
+      this.backIndex[count] = backIndex;
+      this.backID[count] = backID;
+      this.backType[count] = backType;
+      count++;
+    }
+
+    public void reset() {
+      count = 0;
+    }
+  }
+
+  /**
+   * Returns the space penalty associated with the provided {@link POS.Tag}.
+   *
+   * @param leftPOS the left part of speech of the current token.
+   * @param numSpaces the number of spaces before the current token.
+   */
+  private int computeSpacePenalty(POS.Tag leftPOS, int numSpaces) {
+    int spacePenalty = 0;
+    if (numSpaces > 0) {
+      // TODO we should extract the penalty (left-space-penalty-factor) from the dicrc file.
+      switch (leftPOS) {
+        case E:
+        case J:
+        case VCP:
+        case XSA:
+        case XSN:
+        case XSV:
+          spacePenalty = 3000;
+          break;
+
+        default:
+          break;
+      }
+    }
+    return spacePenalty;
+
+  }
+
+  private void add(Dictionary dict, Position fromPosData, int wordPos, int endPos, int wordID, Type type) throws IOException {
+    final POS.Tag leftPOS = dict.getLeftPOS(wordID);
+    final int wordCost = dict.getWordCost(wordID);
+    final int leftID = dict.getLeftId(wordID);
+    int leastCost = Integer.MAX_VALUE;
+    int leastIDX = -1;
+    assert fromPosData.count > 0;
+    for(int idx=0;idx<fromPosData.count;idx++) {
+      // The number of spaces before the term
+      int numSpaces = wordPos - fromPosData.pos;
+
+      // Cost is path cost so far, plus word cost (added at
+      // end of loop), plus bigram cost and space penalty cost.
+      final int cost = fromPosData.costs[idx] + costs.get(fromPosData.lastRightID[idx], leftID) + computeSpacePenalty(leftPOS, numSpaces);
+      if (VERBOSE) {
+        System.out.println("      fromIDX=" + idx + ": cost=" + cost + " (prevCost=" + fromPosData.costs[idx] + " wordCost=" + wordCost + " bgCost=" + costs.get(fromPosData.lastRightID[idx], leftID) +
+            " spacePenalty=" + computeSpacePenalty(leftPOS, numSpaces) + ") leftID=" + leftID + " leftPOS=" + leftPOS.name() + ")");
+      }
+      if (cost < leastCost) {
+        leastCost = cost;
+        leastIDX = idx;
+        if (VERBOSE) {
+          System.out.println("        **");
+        }
+      }
+    }
+
+    leastCost += wordCost;
+
+    if (VERBOSE) {
+      System.out.println("      + cost=" + leastCost + " wordID=" + wordID + " leftID=" + leftID + " leastIDX=" + leastIDX + " toPos=" + endPos + " toPos.idx=" + positions.get(endPos).count);
+    }
+
+    positions.get(endPos).add(leastCost, dict.getRightId(wordID), fromPosData.pos, wordPos, leastIDX, wordID, type);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+
+    // parse() is able to return w/o producing any new
+    // tokens, when the tokens it had produced were entirely
+    // punctuation.  So we loop here until we get a real
+    // token or we end:
+    while (pending.size() == 0) {
+      if (end) {
+        return false;
+      }
+
+      // Push Viterbi forward some more:
+      parse();
+    }
+
+    final Token token = pending.remove(pending.size()-1);
+
+    int length = token.getLength();
+    clearAttributes();
+    assert length > 0;
+    //System.out.println("off=" + token.getOffset() + " len=" + length + " vs " + token.getSurfaceForm().length);
+    termAtt.copyBuffer(token.getSurfaceForm(), token.getOffset(), length);
+    offsetAtt.setOffset(correctOffset(token.getStartOffset()), correctOffset(token.getEndOffset()));
+    posAtt.setToken(token);
+    readingAtt.setToken(token);
+    posIncAtt.setPositionIncrement(token.getPositionIncrement());
+    posLengthAtt.setPositionLength(token.getPositionLength());
+    if (VERBOSE) {
+      System.out.println(Thread.currentThread().getName() + ":    incToken: return token=" + token);
+    }
+    return true;
+  }
+
+  // TODO: make generic'd version of this "circular array"?
+  // It's a bit tricky because we do things to the Position
+  // (eg, set .pos = N on reuse)...
+  static final class WrappedPositionArray {
+    private Position[] positions = new Position[8];
+
+    public WrappedPositionArray() {
+      for(int i=0;i<positions.length;i++) {
+        positions[i] = new Position();
+      }
+    }
+
+    // Next array index to write to in positions:
+    private int nextWrite;
+
+    // Next position to write:
+    private int nextPos;
+
+    // How many valid Position instances are held in the
+    // positions array:
+    private int count;
+
+    public void reset() {
+      nextWrite--;
+      while(count > 0) {
+        if (nextWrite == -1) {
+          nextWrite = positions.length - 1;
+        }
+        positions[nextWrite--].reset();
+        count--;
+      }
+      nextWrite = 0;
+      nextPos = 0;
+      count = 0;
+    }
+
+    /** Get Position instance for this absolute position;
+     *  this is allowed to be arbitrarily far "in the
+     *  future" but cannot be before the last freeBefore. */
+    public Position get(int pos) {
+      while(pos >= nextPos) {
+        //System.out.println("count=" + count + " vs len=" + positions.length);
+        if (count == positions.length) {
+          Position[] newPositions = new Position[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+          //System.out.println("grow positions " + newPositions.length);
+          System.arraycopy(positions, nextWrite, newPositions, 0, positions.length-nextWrite);
+          System.arraycopy(positions, 0, newPositions, positions.length-nextWrite, nextWrite);
+          for(int i=positions.length;i<newPositions.length;i++) {
+            newPositions[i] = new Position();
+          }
+          nextWrite = positions.length;
+          positions = newPositions;
+        }
+        if (nextWrite == positions.length) {
+          nextWrite = 0;
+        }
+        // Should have already been reset:
+        assert positions[nextWrite].count == 0;
+        positions[nextWrite++].pos = nextPos++;
+        count++;
+      }
+      assert inBounds(pos);
+      final int index = getIndex(pos);
+      assert positions[index].pos == pos;
+      return positions[index];
+    }
+
+    public int getNextPos() {
+      return nextPos;
+    }
+
+    // For assert:
+    private boolean inBounds(int pos) {
+      return pos < nextPos && pos >= nextPos - count;
+    }
+
+    private int getIndex(int pos) {
+      int index = nextWrite - (nextPos - pos);
+      if (index < 0) {
+        index += positions.length;
+      }
+      return index;
+    }
+
+    public void freeBefore(int pos) {
+      final int toFree = count - (nextPos - pos);
+      assert toFree >= 0;
+      assert toFree <= count;
+      int index = nextWrite - count;
+      if (index < 0) {
+        index += positions.length;
+      }
+      for(int i=0;i<toFree;i++) {
+        if (index == positions.length) {
+          index = 0;
+        }
+        //System.out.println("  fb idx=" + index);
+        positions[index].reset();
+        index++;
+      }
+      count -= toFree;
+    }
+  }
+
+  /* Incrementally parse some more characters.  This runs
+   * the viterbi search forwards "enough" so that we
+   * generate some more tokens.  How much forward depends on
+   * the chars coming in, since some chars could cause
+   * longer-lasting ambiguity in the parsing.  Once the
+   * ambiguity is resolved, then we back trace, produce
+   * the pending tokens, and return. */
+  private void parse() throws IOException {
+    if (VERBOSE) {
+      System.out.println("\nPARSE");
+    }
+
+    // Index of the last character of unknown word:
+    int unknownWordEndIndex = -1;
+
+    // Advances over each position (character):
+    while (true) {
+
+      if (buffer.get(pos) == -1) {
+        // End
+        break;
+      }
+
+      final Position posData = positions.get(pos);
+      final boolean isFrontier = positions.getNextPos() == pos+1;
+
+      if (posData.count == 0) {
+        // No arcs arrive here; move to next position:
+        if (VERBOSE) {
+          System.out.println("    no arcs in; skip pos=" + pos);
+        }
+        pos++;
+        continue;
+      }
+
+      if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        //  if (pos > lastBackTracePos && posData.count == 1 && isFrontier) {
+        // We are at a "frontier", and only one node is
+        // alive, so whatever the eventual best path is must
+        // come through this node.  So we can safely commit
+        // to the prefix of the best path at this point:
+        backtrace(posData, 0);
+
+        // Re-base cost so we don't risk int overflow:
+        posData.costs[0] = 0;
+        if (pending.size() > 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (pos - lastBackTracePos >= MAX_BACKTRACE_GAP) {
+        // Safety: if we've buffered too much, force a
+        // backtrace now.  We find the least-cost partial
+        // path, across all paths, backtrace from it, and
+        // then prune all others.  Note that this, in
+        // general, can produce the wrong result, if the
+        // total best path did not in fact back trace
+        // through this partial best path.  But it's the
+        // best we can do... (short of not having a
+        // safety!).
+
+        // First pass: find least cost partial path so far,
+        // including ending at future positions:
+        int leastIDX = -1;
+        int leastCost = Integer.MAX_VALUE;
+        Position leastPosData = null;
+        for(int pos2=pos;pos2<positions.getNextPos();pos2++) {
+          final Position posData2 = positions.get(pos2);
+          for(int idx=0;idx<posData2.count;idx++) {
+            //System.out.println("    idx=" + idx + " cost=" + cost);
+            final int cost = posData2.costs[idx];
+            if (cost < leastCost) {
+              leastCost = cost;
+              leastIDX = idx;
+              leastPosData = posData2;
+            }
+          }
+        }
+
+        // We will always have at least one live path:
+        assert leastIDX != -1;
+
+        // Second pass: prune all but the best path:
+        for(int pos2=pos;pos2<positions.getNextPos();pos2++) {
+          final Position posData2 = positions.get(pos2);
+          if (posData2 != leastPosData) {
+            posData2.reset();
+          } else {
+            if (leastIDX != 0) {
+              posData2.costs[0] = posData2.costs[leastIDX];
+              posData2.lastRightID[0] = posData2.lastRightID[leastIDX];
+              posData2.backPos[0] = posData2.backPos[leastIDX];
+              posData2.backWordPos[0] = posData2.backWordPos[leastIDX];
+              posData2.backIndex[0] = posData2.backIndex[leastIDX];
+              posData2.backID[0] = posData2.backID[leastIDX];
+              posData2.backType[0] = posData2.backType[leastIDX];
+            }
+            posData2.count = 1;
+          }
+        }
+
+        backtrace(leastPosData, 0);
+
+        // Re-base cost so we don't risk int overflow:
+        Arrays.fill(leastPosData.costs, 0, leastPosData.count, 0);
+
+        if (pos != leastPosData.pos) {
+          // We jumped into a future position:
+          assert pos < leastPosData.pos;
+          pos = leastPosData.pos;
+        }
+        if (pending.size() > 0) {
+          return;
+        } else {
+          // This means the backtrace only produced
+          // punctuation tokens, so we must keep parsing.
+        }
+      }
+
+      if (VERBOSE) {
+        System.out.println("\n  extend @ pos=" + pos + " char=" + (char) buffer.get(pos) + " hex=" + Integer.toHexString(buffer.get(pos)));
+      }
+
+      if (VERBOSE) {
+        System.out.println("    " + posData.count + " arcs in");
+      }
+
+      // Move to the first character that is not a whitespace.
+      // The whitespaces are added as a prefix for the term that we extract,
+      // this information is then used when computing the cost for the term using
+      // the space penalty factor.
+      // They are removed when the final tokens are generated.
+      if (Character.getType(buffer.get(pos)) == Character.SPACE_SEPARATOR) {
+        int nextChar = buffer.get(++pos);
+        while (nextChar != -1 && Character.getType(nextChar) == Character.SPACE_SEPARATOR) {
+          pos ++;
+          nextChar = buffer.get(pos);
+        }
+      }
+      if (buffer.get(pos) == -1) {
+        pos = posData.pos;
+      }
+
+      boolean anyMatches = false;
+
+      // First try user dict:
+      if (userFST != null) {
+        userFST.getFirstArc(arc);
+        int output = 0;
+        for(int posAhead=pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          if (userFST.findTargetArc(ch, arc, arc, posAhead == pos, userFSTReader) == null) {
+            break;
+          }
+          output += arc.output.intValue();
+          if (arc.isFinal()) {
+            if (VERBOSE) {
+              System.out.println("    USER word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1));
+            }
+            add(userDictionary, posData, pos, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER);
+            anyMatches = true;
+          }
+        }
+      }
+
+      // TODO: we can be more aggressive about user
+      // matches?  if we are "under" a user match then don't
+      // extend KNOWN/UNKNOWN paths?
+
+      if (!anyMatches) {
+        // Next, try known dictionary matches
+        fst.getFirstArc(arc);
+        int output = 0;
+
+        for(int posAhead=pos;;posAhead++) {
+          final int ch = buffer.get(posAhead);
+          if (ch == -1) {
+            break;
+          }
+          //System.out.println("    match " + (char) ch + " posAhead=" + posAhead);
+
+          if (fst.findTargetArc(ch, arc, arc, posAhead == pos, fstReader) == null) {
+            break;
+          }
+
+          output += arc.output.intValue();
+
+          // Optimization: for known words that are too-long
+          // (compound), we should pre-compute the 2nd
+          // best segmentation and store it in the
+          // dictionary instead of recomputing it each time a
+          // match is found.
+
+          if (arc.isFinal()) {
+            dictionary.lookupWordIds(output + arc.nextFinalOutput.intValue(), wordIdRef);
+            if (VERBOSE) {
+              System.out.println("    KNOWN word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1) + " " + wordIdRef.length + " wordIDs");
+            }
+            for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+              add(dictionary, posData, pos, posAhead+1, wordIdRef.ints[wordIdRef.offset + ofs], Type.KNOWN);
+              anyMatches = true;
+            }
+          }
+        }
+      }
+
+      if (unknownWordEndIndex > posData.pos) {
+        pos++;
+        continue;
+      }
+
+      final char firstCharacter = (char) buffer.get(pos);
+      if (!anyMatches || characterDefinition.isInvoke(firstCharacter)) {
+
+        // Find unknown match:
+        final int characterId = characterDefinition.getCharacterClass(firstCharacter);
+        final boolean isPunct = isPunctuation(firstCharacter);
+
+        // NOTE: copied from UnknownDictionary.lookup:
+        int unknownWordLength;
+        if (!characterDefinition.isGroup(firstCharacter)) {
+          unknownWordLength = 1;
+        } else {
+          // Extract unknown word. Characters with the same character class are considered to be part of unknown word
+          unknownWordLength = 1;
+          for (int posAhead = pos + 1; unknownWordLength < MAX_UNKNOWN_WORD_LENGTH; posAhead++) {
+            final int ch = buffer.get(posAhead);
+            if (ch == -1) {
+              break;
+            }
+            if (characterId == characterDefinition.getCharacterClass((char) ch) &&
+                isPunctuation((char) ch) == isPunct) {
+              unknownWordLength++;
+            } else {
+              break;
+            }
+          }
+        }
+
+        unkDictionary.lookupWordIds(characterId, wordIdRef); // characters in input text are supposed to be the same
+        if (VERBOSE) {
+          System.out.println("    UNKNOWN word len=" + unknownWordLength + " " + wordIdRef.length + " wordIDs");
+        }
+        for (int ofs = 0; ofs < wordIdRef.length; ofs++) {
+          add(unkDictionary, posData, pos, pos + unknownWordLength, wordIdRef.ints[wordIdRef.offset + ofs], Type.UNKNOWN);
+        }
+      }
+
+      pos++;
+    }
+
+    end = true;
+
+    if (pos > 0) {
+
+      final Position endPosData = positions.get(pos);
+      int leastCost = Integer.MAX_VALUE;
+      int leastIDX = -1;
+      if (VERBOSE) {
+        System.out.println("  end: " + endPosData.count + " nodes");
+      }
+      for(int idx=0;idx<endPosData.count;idx++) {
+        // Add EOS cost:
+        final int cost = endPosData.costs[idx] + costs.get(endPosData.lastRightID[idx], 0);
+        //System.out.println("    idx=" + idx + " cost=" + cost + " (pathCost=" + endPosData.costs[idx] + " bgCost=" + costs.get(endPosData.lastRightID[idx], 0) + ") backPos=" + endPosData.backPos[idx]);
+        if (cost < leastCost) {
+          leastCost = cost;
+          leastIDX = idx;
+        }
+      }
+
+      backtrace(endPosData, leastIDX);
+    } else {
+      // No characters in the input string; return no tokens!
+    }
+  }
+
+  // the pending list.  The pending list is then in-reverse
+  // (last token should be returned first).
+  private void backtrace(final Position endPosData, final int fromIDX) {
+    final int endPos = endPosData.pos;
+
+    if (VERBOSE) {
+      System.out.println("\n  backtrace: endPos=" + endPos + " pos=" + pos + "; " + (pos - lastBackTracePos) + " characters; last=" + lastBackTracePos + " cost=" + endPosData.costs[fromIDX]);
+    }
+
+    final char[] fragment = buffer.get(lastBackTracePos, endPos-lastBackTracePos);
+
+    if (dotOut != null) {
+      dotOut.onBacktrace(this, positions, lastBackTracePos, endPosData, fromIDX, fragment, end);
+    }
+
+    int pos = endPos;
+    int bestIDX = fromIDX;
+
+    // TODO: sort of silly to make Token instances here; the
+    // back trace has all info needed to generate the
+    // token.  So, we could just directly set the attrs,
+    // from the backtrace, in incrementToken w/o ever
+    // creating Token; we'd have to defer calling freeBefore
+    // until after the backtrace was fully "consumed" by
+    // incrementToken.
+
+    while (pos > lastBackTracePos) {
+      //System.out.println("BT: back pos=" + pos + " bestIDX=" + bestIDX);
+      final Position posData = positions.get(pos);
+      assert bestIDX < posData.count;
+
+      int backPos = posData.backPos[bestIDX];
+      int backWordPos = posData.backWordPos[bestIDX];
+      assert backPos >= lastBackTracePos: "backPos=" + backPos + " vs lastBackTracePos=" + lastBackTracePos;
+      // the length of the word without the whitespaces at the beginning.
+      int length = pos - backWordPos;
+      Type backType = posData.backType[bestIDX];
+      int backID = posData.backID[bestIDX];
+      int nextBestIDX = posData.backIndex[bestIDX];
+      // the start of the word after the whitespace at the beginning.
+      final int fragmentOffset = backWordPos - lastBackTracePos;
+      assert fragmentOffset >= 0;
+
+      final Dictionary dict = getDict(backType);
+
+      if (outputUnknownUnigrams && backType == Type.UNKNOWN) {
+        // outputUnknownUnigrams converts unknown word into unigrams:
+        for (int i = length - 1; i >= 0; i--) {
+          int charLen = 1;
+          if (i > 0 && Character.isLowSurrogate(fragment[fragmentOffset + i])) {
+            i--;
+            charLen = 2;
+          }
+          final DictionaryToken token = new DictionaryToken(Type.UNKNOWN,
+              unkDictionary,
+              CharacterDefinition.NGRAM,
+              fragment,
+              fragmentOffset+i,
+              charLen,
+              backWordPos+i,
+              backWordPos+i+charLen
+          );
+          if (shouldFilterToken(token) == false) {
+            pending.add(token);
+            if (VERBOSE) {
+              System.out.println("    add token=" + pending.get(pending.size() - 1));
+            }
+          }
+        }
+      } else {
+        final DictionaryToken token = new DictionaryToken(backType,
+            dict,
+            backID,
+            fragment,
+            fragmentOffset,
+            length,
+            backWordPos,
+            backWordPos + length
+        );
+        if (token.getPOSType() == POS.Type.MORPHEME || mode == DecompoundMode.NONE) {
+          if (shouldFilterToken(token) == false) {
+            pending.add(token);
+            if (VERBOSE) {
+              System.out.println("    add token=" + pending.get(pending.size() - 1));
+            }
+          }
+        } else {
+          Dictionary.Morpheme[] morphemes = token.getMorphemes();
+          if (morphemes == null) {
+            pending.add(token);
+            if (VERBOSE) {
+              System.out.println("    add token=" + pending.get(pending.size() - 1));
+            }
+          } else {
+            int endOffset = backWordPos + length;
+            int posLen = 0;
+            // decompose the compound
+            for (int i = morphemes.length - 1; i >= 0; i--) {
+              final Dictionary.Morpheme morpheme = morphemes[i];
+              final Token compoundToken;
+              if (token.getPOSType() == POS.Type.COMPOUND) {
+                assert endOffset - morpheme.surfaceForm.length() >= 0;
+                compoundToken = new DecompoundToken(morpheme.posTag, morpheme.surfaceForm,
+                    endOffset - morpheme.surfaceForm.length(), endOffset);
+              } else {
+                compoundToken = new DecompoundToken(morpheme.posTag, morpheme.surfaceForm, token.getStartOffset(), token.getEndOffset());
+              }
+              if (i == 0 && mode == DecompoundMode.MIXED) {
+                compoundToken.setPositionIncrement(0);
+              }
+              ++ posLen;
+              endOffset -= morpheme.surfaceForm.length();
+              pending.add(compoundToken);
+              if (VERBOSE) {
+                System.out.println("    add token=" + pending.get(pending.size() - 1));
+              }
+            }
+            if (mode == DecompoundMode.MIXED) {
+              token.setPositionLength(Math.max(1, posLen));
+              pending.add(token);
+              if (VERBOSE) {
+                System.out.println("    add token=" + pending.get(pending.size() - 1));
+              }
+            }
+          }
+        }
+      }
+
+      pos = backPos;
+      bestIDX = nextBestIDX;
+    }
+
+    lastBackTracePos = endPos;
+
+    if (VERBOSE) {
+      System.out.println("  freeBefore pos=" + endPos);
+    }
+    // Notify the circular buffers that we are done with
+    // these positions:
+    buffer.freeBefore(endPos);
+    positions.freeBefore(endPos);
+  }
+
+  Dictionary getDict(Type type) {
+    return dictionaryMap.get(type);
+  }
+
+  private boolean shouldFilterToken(Token token) {
+    return isPunctuation(token.getSurfaceForm()[token.getOffset()]);
+  }
+
+  private static boolean isPunctuation(char ch) {
+    switch(Character.getType(ch)) {
+      case Character.SPACE_SEPARATOR:
+      case Character.LINE_SEPARATOR:
+      case Character.PARAGRAPH_SEPARATOR:
+      case Character.CONTROL:
+      case Character.FORMAT:
+      case Character.DASH_PUNCTUATION:
+      case Character.START_PUNCTUATION:
+      case Character.END_PUNCTUATION:
+      case Character.CONNECTOR_PUNCTUATION:
+      case Character.OTHER_PUNCTUATION:
+      case Character.MATH_SYMBOL:
+      case Character.CURRENCY_SYMBOL:
+      case Character.MODIFIER_SYMBOL:
+      case Character.OTHER_SYMBOL:
+      case Character.INITIAL_QUOTE_PUNCTUATION:
+      case Character.FINAL_QUOTE_PUNCTUATION:
+        return true;
+      default:
+        return false;
+    }
+  }
+}


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

Posted by ji...@apache.org.
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);
+  }
+}