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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8651fbaa/lucene/analysis/nori/src/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));
+  }
+}