You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ni...@apache.org on 2017/07/23 17:36:33 UTC

[08/13] lucenenet git commit: Ported Lucene.Net.Analysis.Kuromoji + tests

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/BaseFormAttributeImpl.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/BaseFormAttributeImpl.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/BaseFormAttributeImpl.cs
new file mode 100644
index 0000000..7121c73
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/BaseFormAttributeImpl.cs
@@ -0,0 +1,55 @@
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for <see cref="Token.GetBaseForm()"/>.
+    /// </summary>
+    public class BaseFormAttribute : Attribute, IBaseFormAttribute
+    {
+        private Token token;
+
+        public virtual string GetBaseForm()
+        {
+            return token == null ? null : token.GetBaseForm();
+        }
+
+        public virtual void SetToken(Token token)
+        {
+            this.token = token;
+        }
+
+        public override void Clear()
+        {
+            token = null;
+        }
+
+        public override void CopyTo(IAttribute target)
+        {
+            BaseFormAttribute t = (BaseFormAttribute)target;
+            t.SetToken(token);
+        }
+
+        public override void ReflectWith(IAttributeReflector reflector)
+        {
+            reflector.Reflect(typeof(BaseFormAttribute), "baseForm", GetBaseForm());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttribute.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttribute.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttribute.cs
new file mode 100644
index 0000000..975d8b0
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttribute.cs
@@ -0,0 +1,34 @@
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for Kuromoji inflection data.
+    /// <para/>
+    /// Note: in some cases this value may not be applicable,
+    /// and will be null.
+    /// </summary>
+    public interface IInflectionAttribute : IAttribute
+    {
+        string GetInflectionType();
+        string GetInflectionForm();
+        void SetToken(Token token);
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttributeImpl.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttributeImpl.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttributeImpl.cs
new file mode 100644
index 0000000..db96130
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/InflectionAttributeImpl.cs
@@ -0,0 +1,68 @@
+using Lucene.Net.Analysis.Ja.Util;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for Kuromoji inflection data.
+    /// </summary>
+    public class InflectionAttribute : Attribute, IInflectionAttribute
+    {
+        private Token token;
+
+        public virtual string GetInflectionType()
+        {
+            return token == null ? null : token.GetInflectionType();
+        }
+
+        public virtual string GetInflectionForm()
+        {
+            return token == null ? null : token.GetInflectionForm();
+        }
+
+        public virtual void SetToken(Token token)
+        {
+            this.token = token;
+        }
+
+        public override void Clear()
+        {
+            token = null;
+        }
+
+        public override void CopyTo(IAttribute target)
+        {
+            InflectionAttribute t = (InflectionAttribute)target;
+            t.SetToken(token);
+        }
+
+        public override void ReflectWith(IAttributeReflector reflector)
+        {
+            string type = GetInflectionType();
+            string typeEN = type == null ? null : ToStringUtil.GetInflectionTypeTranslation(type);
+            reflector.Reflect<IInflectionAttribute>("inflectionType", type);
+            reflector.Reflect<IInflectionAttribute>("inflectionType (en)", typeEN);
+            string form = GetInflectionForm();
+            string formEN = form == null ? null : ToStringUtil.GetInflectedFormTranslation(form);
+            reflector.Reflect<IInflectionAttribute>("inflectionForm", form);
+            reflector.Reflect<IInflectionAttribute>("inflectionForm (en)", formEN);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttribute.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttribute.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttribute.cs
new file mode 100644
index 0000000..722d203
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttribute.cs
@@ -0,0 +1,30 @@
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for <see cref="Token.GetPartOfSpeech()"/>.
+    /// </summary>
+    public interface IPartOfSpeechAttribute : IAttribute
+    {
+        string GetPartOfSpeech();
+        void SetToken(Token token);
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttributeImpl.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttributeImpl.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttributeImpl.cs
new file mode 100644
index 0000000..9d0451d
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/PartOfSpeechAttributeImpl.cs
@@ -0,0 +1,59 @@
+using Lucene.Net.Analysis.Ja.Util;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for <see cref="Token.GetPartOfSpeech()"/>.
+    /// </summary>
+    public class PartOfSpeechAttribute : Attribute, IPartOfSpeechAttribute
+    {
+        private Token token;
+
+        public virtual string GetPartOfSpeech()
+        {
+            return token == null ? null : token.GetPartOfSpeech();
+        }
+
+        public virtual void SetToken(Token token)
+        {
+            this.token = token;
+        }
+
+        public override void Clear()
+        {
+            token = null;
+        }
+
+        public override void CopyTo(IAttribute target)
+        {
+            PartOfSpeechAttribute t = (PartOfSpeechAttribute)target;
+            t.SetToken(token);
+        }
+
+        public override void ReflectWith(IAttributeReflector reflector)
+        {
+            string partOfSpeech = GetPartOfSpeech();
+            string partOfSpeechEN = partOfSpeech == null ? null : ToStringUtil.GetPOSTranslation(partOfSpeech);
+            reflector.Reflect<IPartOfSpeechAttribute>("partOfSpeech", partOfSpeech);
+            reflector.Reflect<IPartOfSpeechAttribute>("partOfSpeech (en)", partOfSpeechEN);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttribute.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttribute.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttribute.cs
new file mode 100644
index 0000000..2432872
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttribute.cs
@@ -0,0 +1,34 @@
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for Kuromoji reading data
+    /// <para/>
+    /// Note: in some cases this value may not be applicable,
+    /// and will be null.
+    /// </summary>
+    public interface IReadingAttribute : IAttribute
+    {
+        string GetReading();
+        string GetPronunciation();
+        void SetToken(Token token);
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttributeImpl.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttributeImpl.cs b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttributeImpl.cs
new file mode 100644
index 0000000..bfb8a93
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/TokenAttributes/ReadingAttributeImpl.cs
@@ -0,0 +1,68 @@
+using Lucene.Net.Analysis.Ja.Util;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Analysis.Ja.TokenAttributes
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Attribute for Kuromoji reading data
+    /// </summary>
+    public class ReadingAttribute : Attribute, IReadingAttribute
+    {
+        private Token token;
+
+        public virtual string GetReading()
+        {
+            return token == null ? null : token.GetReading();
+        }
+
+        public virtual string GetPronunciation()
+        {
+            return token == null ? null : token.GetPronunciation();
+        }
+
+        public virtual void SetToken(Token token)
+        {
+            this.token = token;
+        }
+
+        public override void Clear()
+        {
+            token = null;
+        }
+
+        public override void CopyTo(IAttribute target)
+        {
+            ReadingAttribute t = (ReadingAttribute)target;
+            t.SetToken(token);
+        }
+
+        public override void ReflectWith(IAttributeReflector reflector)
+        {
+            string reading = GetReading();
+            string readingEN = reading == null ? null : ToStringUtil.GetRomanization(reading);
+            string pronunciation = GetPronunciation();
+            string pronunciationEN = pronunciation == null ? null : ToStringUtil.GetRomanization(pronunciation);
+            reflector.Reflect<IReadingAttribute>("reading", reading);
+            reflector.Reflect<IReadingAttribute>("reading (en)", readingEN);
+            reflector.Reflect<IReadingAttribute>("pronunciation", pronunciation);
+            reflector.Reflect<IReadingAttribute>("pronunciation (en)", pronunciationEN);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs
new file mode 100644
index 0000000..39d36eb
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs
@@ -0,0 +1,370 @@
+using Lucene.Net.Analysis.Ja.Dict;
+using Lucene.Net.Codecs;
+using Lucene.Net.Store;
+using Lucene.Net.Support.IO;
+using Lucene.Net.Util;
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Globalization;
+using System.IO;
+using System.Text;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public abstract class BinaryDictionaryWriter
+    {
+        protected readonly Type m_implClazz;
+        protected ByteBuffer m_buffer;
+        private int targetMapEndOffset = 0, lastWordId = -1, lastSourceId = -1;
+        private int[] targetMap = new int[8192];
+        private int[] targetMapOffsets = new int[8192];
+        private readonly List<string> posDict = new List<string>();
+
+        public BinaryDictionaryWriter(Type implClazz, int size)
+        {
+            this.m_implClazz = implClazz;
+            m_buffer = ByteBuffer.Allocate(size);
+        }
+
+        /// <summary>
+        /// Put the entry in map.
+        /// </summary>
+        /// <param name="entry"></param>
+        /// <returns>Current position of buffer, which will be wordId of next entry.</returns>
+        public virtual int Put(string[] entry)
+        {
+            short leftId = short.Parse(entry[1], CultureInfo.InvariantCulture);
+            short rightId = short.Parse(entry[2], CultureInfo.InvariantCulture);
+            short wordCost = short.Parse(entry[3], CultureInfo.InvariantCulture);
+
+            StringBuilder sb = new StringBuilder();
+
+            // build up the POS string
+            for (int i = 4; i < 8; i++)
+            {
+                string part = entry[i];
+                Debug.Assert(part.Length > 0);
+                if (!"*".Equals(part, StringComparison.Ordinal))
+                {
+                    if (sb.Length > 0)
+                    {
+                        sb.Append('-');
+                    }
+                    sb.Append(part);
+                }
+            }
+
+            string posData = sb.ToString();
+
+            sb.Length = 0;
+            sb.Append(CSVUtil.QuoteEscape(posData));
+            sb.Append(',');
+            if (!"*".Equals(entry[8]))
+            {
+                sb.Append(CSVUtil.QuoteEscape(entry[8]));
+            }
+            sb.Append(',');
+            if (!"*".Equals(entry[9]))
+            {
+                sb.Append(CSVUtil.QuoteEscape(entry[9]));
+            }
+            string fullPOSData = sb.ToString();
+
+            string baseForm = entry[10];
+            string reading = entry[11];
+            string pronunciation = entry[12];
+
+            // extend buffer if necessary
+            int left = m_buffer.Remaining;
+            // worst case: two short, 3 bytes, and features (all as utf-16)
+            int worstCase = 4 + 3 + 2 * (baseForm.Length + reading.Length + pronunciation.Length);
+            if (worstCase > left)
+            {
+                ByteBuffer newBuffer = ByteBuffer.Allocate(ArrayUtil.Oversize(m_buffer.Limit + worstCase - left, 1));
+                m_buffer.Flip();
+                newBuffer.Put(m_buffer);
+                m_buffer = newBuffer;
+            }
+
+            int flags = 0;
+            if (!("*".Equals(baseForm) || baseForm.Equals(entry[0])))
+            {
+                flags |= BinaryDictionary.HAS_BASEFORM;
+            }
+            if (!reading.Equals(ToKatakana(entry[0])))
+            {
+                flags |= BinaryDictionary.HAS_READING;
+            }
+            if (!pronunciation.Equals(reading))
+            {
+                flags |= BinaryDictionary.HAS_PRONUNCIATION;
+            }
+
+            Debug.Assert(leftId == rightId);
+            Debug.Assert(leftId < 4096); // there are still unused bits
+                                         // add pos mapping
+            int toFill = 1 + leftId - posDict.Count;
+            for (int i = 0; i < toFill; i++)
+            {
+                posDict.Add(null);
+            }
+
+            string existing = posDict[leftId];
+            Debug.Assert(existing == null || existing.Equals(fullPOSData));
+            posDict[leftId] = fullPOSData;
+
+            m_buffer.PutInt16((short)(leftId << 3 | flags));
+            m_buffer.PutInt16(wordCost);
+
+            if ((flags & BinaryDictionary.HAS_BASEFORM) != 0)
+            {
+                Debug.Assert(baseForm.Length < 16);
+                int shared = SharedPrefix(entry[0], baseForm);
+                int suffix = baseForm.Length - shared;
+                m_buffer.Put((byte)(shared << 4 | suffix));
+                for (int i = shared; i < baseForm.Length; i++)
+                {
+                    m_buffer.PutChar(baseForm[i]);
+                }
+            }
+
+            if ((flags & BinaryDictionary.HAS_READING) != 0)
+            {
+                if (IsKatakana(reading))
+                {
+                    m_buffer.Put((byte)(reading.Length << 1 | 1));
+                    WriteKatakana(reading);
+                }
+                else
+                {
+                    m_buffer.Put((byte)(reading.Length << 1));
+                    for (int i = 0; i < reading.Length; i++)
+                    {
+                        m_buffer.PutChar(reading[i]);
+                    }
+                }
+            }
+
+            if ((flags & BinaryDictionary.HAS_PRONUNCIATION) != 0)
+            {
+                // we can save 150KB here, but it makes the reader a little complicated.
+                // int shared = sharedPrefix(reading, pronunciation);
+                // buffer.put((byte) shared);
+                // pronunciation = pronunciation.substring(shared);
+                if (IsKatakana(pronunciation))
+                {
+                    m_buffer.Put((byte)(pronunciation.Length << 1 | 1));
+                    WriteKatakana(pronunciation);
+                }
+                else
+                {
+                    m_buffer.Put((byte)(pronunciation.Length << 1));
+                    for (int i = 0; i < pronunciation.Length; i++)
+                    {
+                        m_buffer.PutChar(pronunciation[i]);
+                    }
+                }
+            }
+
+            return m_buffer.Position;
+        }
+
+        private bool IsKatakana(string s)
+        {
+            for (int i = 0; i < s.Length; i++)
+            {
+                char ch = s[i];
+                if (ch < 0x30A0 || ch > 0x30FF)
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        private void WriteKatakana(string s)
+        {
+            for (int i = 0; i < s.Length; i++)
+            {
+                m_buffer.Put((byte)(s[i] - 0x30A0));
+            }
+        }
+
+        private string ToKatakana(string s)
+        {
+            char[] text = new char[s.Length];
+            for (int i = 0; i < s.Length; i++)
+            {
+                char ch = s[i];
+                if (ch > 0x3040 && ch < 0x3097)
+                {
+                    text[i] = (char)(ch + 0x60);
+                }
+                else
+                {
+                    text[i] = ch;
+                }
+            }
+            return new string(text);
+        }
+
+        public static int SharedPrefix(string left, string right)
+        {
+            int len = left.Length < right.Length ? left.Length : right.Length;
+            for (int i = 0; i < len; i++)
+                if (left[i] != right[i])
+                    return i;
+            return len;
+        }
+
+        public virtual void AddMapping(int sourceId, int wordId)
+        {
+            Debug.Assert(wordId > lastWordId, "words out of order: " + wordId + " vs lastID: " + lastWordId);
+
+            if (sourceId > lastSourceId)
+            {
+                Debug.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
+            {
+                Debug.Assert(sourceId == lastSourceId);
+            }
+
+            targetMap = ArrayUtil.Grow(targetMap, targetMapEndOffset + 1);
+            targetMap[targetMapEndOffset] = wordId;
+            targetMapEndOffset++;
+
+            lastSourceId = sourceId;
+            lastWordId = wordId;
+        }
+
+        protected string GetBaseFileName(string baseDir)
+        {
+            return baseDir + System.IO.Path.DirectorySeparatorChar + m_implClazz.FullName.Replace('.', System.IO.Path.DirectorySeparatorChar);
+        }
+
+        /// <summary>
+        /// Write dictionary in file
+        /// </summary>
+        /// <remarks>
+        /// Dictionary format is:
+        /// [Size of dictionary(int)], [entry:{left id(short)}{right id(short)}{word cost(short)}{length of pos info(short)}{pos info(char)}], [entry...], [entry...].....
+        /// </remarks>
+        /// <param name="baseDir"></param>
+        /// <exception cref="IOException">If an I/O error occurs writing the dictionary files.</exception>
+        public virtual void Write(string baseDir)
+        {
+            string baseName = GetBaseFileName(baseDir);
+            WriteDictionary(baseName + BinaryDictionary.DICT_FILENAME_SUFFIX);
+            WriteTargetMap(baseName + BinaryDictionary.TARGETMAP_FILENAME_SUFFIX);
+            WritePosDict(baseName + BinaryDictionary.POSDICT_FILENAME_SUFFIX);
+        }
+
+        // TODO: maybe this int[] should instead be the output to the FST...
+        protected virtual void WriteTargetMap(string filename)
+        {
+            //new File(filename).getParentFile().mkdirs();
+            System.IO.Directory.CreateDirectory(System.IO.Path.GetDirectoryName(filename));
+            using (Stream os = new FileStream(filename, FileMode.Create, FileAccess.Write))
+            {
+                DataOutput @out = new OutputStreamDataOutput(os);
+                CodecUtil.WriteHeader(@out, BinaryDictionary.TARGETMAP_HEADER, BinaryDictionary.VERSION);
+
+                int numSourceIds = lastSourceId + 1;
+                @out.WriteVInt32(targetMapEndOffset); // <-- size of main array
+                @out.WriteVInt32(numSourceIds + 1); // <-- size of offset array (+ 1 more entry)
+                int prev = 0, sourceId = 0;
+                for (int ofs = 0; ofs < targetMapEndOffset; ofs++)
+                {
+                    int val = targetMap[ofs], delta = val - prev;
+                    Debug.Assert(delta >= 0);
+                    if (ofs == targetMapOffsets[sourceId])
+                    {
+                        @out.WriteVInt32((delta << 1) | 0x01);
+                        sourceId++;
+                    }
+                    else
+                    {
+                        @out.WriteVInt32((delta << 1));
+                    }
+                    prev += delta;
+                }
+                Debug.Assert(sourceId == numSourceIds, "sourceId:" + sourceId + " != numSourceIds:" + numSourceIds);
+            }
+        }
+
+        protected virtual void WritePosDict(string filename)
+        {
+            //new File(filename).getParentFile().mkdirs();
+            System.IO.Directory.CreateDirectory(System.IO.Path.GetDirectoryName(filename));
+            using (Stream os = new FileStream(filename, FileMode.Create, FileAccess.Write))
+            {
+                DataOutput @out = new OutputStreamDataOutput(os);
+                CodecUtil.WriteHeader(@out, BinaryDictionary.POSDICT_HEADER, BinaryDictionary.VERSION);
+                @out.WriteVInt32(posDict.Count);
+                foreach (string s in posDict)
+                {
+                    if (s == null)
+                    {
+                        @out.WriteByte((byte)0);
+                        @out.WriteByte((byte)0);
+                        @out.WriteByte((byte)0);
+                    }
+                    else
+                    {
+                        string[] data = CSVUtil.Parse(s);
+                        Debug.Assert(data.Length == 3, "malformed pos/inflection: " + s);
+                        @out.WriteString(data[0]);
+                        @out.WriteString(data[1]);
+                        @out.WriteString(data[2]);
+                    }
+                }
+            }
+        }
+
+        protected virtual void WriteDictionary(string filename)
+        {
+            //new File(filename).getParentFile().mkdirs();
+            System.IO.Directory.CreateDirectory(System.IO.Path.GetDirectoryName(filename));
+            using (Stream os = new FileStream(filename, FileMode.Create, FileAccess.Write))
+            {
+                DataOutput @out = new OutputStreamDataOutput(os);
+                CodecUtil.WriteHeader(@out, BinaryDictionary.DICT_HEADER, BinaryDictionary.VERSION);
+                @out.WriteVInt32(m_buffer.Position);
+                var writer = new BinaryWriter(os);
+
+                //WritableByteChannel channel = Channels.newChannel(os);
+                // Write Buffer
+                m_buffer.Flip();  // set position to 0, set limit to current position
+                //channel.write(buffer);
+
+                writer.Write(m_buffer.Array, m_buffer.Position, m_buffer.Limit);
+
+                Debug.Assert(m_buffer.Remaining == 0L);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs
new file mode 100644
index 0000000..6d5526d
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs
@@ -0,0 +1,91 @@
+using Lucene.Net.Analysis.Ja.Dict;
+using Lucene.Net.Codecs;
+using Lucene.Net.Store;
+using Lucene.Net.Support;
+using System;
+using System.IO;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public sealed class CharacterDefinitionWriter
+    {
+        private readonly byte[] characterCategoryMap = new byte[0x10000];
+
+        private readonly bool[] invokeMap = new bool[CharacterDefinition.CLASS_COUNT];
+        private readonly bool[] groupMap = new bool[CharacterDefinition.CLASS_COUNT];
+
+        /// <summary>
+        /// Constructor for building. TODO: remove write access
+        /// </summary>
+        public CharacterDefinitionWriter()
+        {
+            Arrays.Fill(characterCategoryMap, CharacterDefinition.DEFAULT);
+        }
+
+        /// <summary>
+        /// Put mapping from unicode code point to character class.
+        /// </summary>
+        /// <param name="codePoint">Code point.</param>
+        /// <param name="characterClassName">Character class name.</param>
+        public void PutCharacterCategory(int codePoint, string characterClassName)
+        {
+            characterClassName = characterClassName.Split(new char[] { ' ' }, StringSplitOptions.RemoveEmptyEntries)[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)
+        {
+            byte characterClass = CharacterDefinition.LookupCharacterClass(characterClassName);
+            invokeMap[characterClass] = invoke == 1;
+            groupMap[characterClass] = group == 1;
+            // TODO: length def ignored
+        }
+
+        public void Write(string baseDir)
+        {
+            string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
+                typeof(CharacterDefinition).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + CharacterDefinition.FILENAME_SUFFIX;
+            //new File(filename).getParentFile().mkdirs();
+            System.IO.Directory.CreateDirectory(System.IO.Path.GetDirectoryName(baseDir));
+            using (Stream os = new FileStream(filename, FileMode.Create, FileAccess.Write))
+            {
+                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++)
+                {
+                    byte b = (byte)(
+                      (invokeMap[i] ? 0x01 : 0x00) |
+                      (groupMap[i] ? 0x02 : 0x00)
+                    );
+                    @out.WriteByte(b);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsBuilder.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsBuilder.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsBuilder.cs
new file mode 100644
index 0000000..235c523
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsBuilder.cs
@@ -0,0 +1,68 @@
+using System.Diagnostics;
+using System.Globalization;
+using System.IO;
+using System.Text;
+using System.Text.RegularExpressions;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class ConnectionCostsBuilder
+    {
+        private static readonly Regex whiteSpaceRegex = new Regex("\\s+", RegexOptions.Compiled);
+
+        private ConnectionCostsBuilder()
+        {
+        }
+
+        public static ConnectionCostsWriter Build(string filename)
+        {
+            using (Stream inputStream = new FileStream(filename, FileMode.Open, FileAccess.Read))
+            {
+                StreamReader streamReader = new StreamReader(inputStream, Encoding.ASCII);
+
+                string line = streamReader.ReadLine();
+                string[] dimensions = whiteSpaceRegex.Split(line);
+
+                Debug.Assert(dimensions.Length == 2);
+
+                int forwardSize = int.Parse(dimensions[0], CultureInfo.InvariantCulture);
+                int backwardSize = int.Parse(dimensions[1], CultureInfo.InvariantCulture);
+
+                Debug.Assert(forwardSize > 0 && backwardSize > 0);
+
+                ConnectionCostsWriter costs = new ConnectionCostsWriter(forwardSize, backwardSize);
+
+                while ((line = streamReader.ReadLine()) != null)
+                {
+                    string[] fields = whiteSpaceRegex.Split(line);
+
+                    Debug.Assert(fields.Length == 3);
+
+                    int forwardId = int.Parse(fields[0], CultureInfo.InvariantCulture);
+                    int backwardId = int.Parse(fields[1], CultureInfo.InvariantCulture);
+                    int cost = int.Parse(fields[2], CultureInfo.InvariantCulture);
+
+                    costs.Add(forwardId, backwardId, cost);
+                }
+                return costs;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs
new file mode 100644
index 0000000..bd1376e
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs
@@ -0,0 +1,74 @@
+using Lucene.Net.Analysis.Ja.Dict;
+using Lucene.Net.Codecs;
+using Lucene.Net.Store;
+using System.Diagnostics;
+using System.IO;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public sealed class ConnectionCostsWriter
+    {
+        private readonly short[][] costs; // array is backward IDs first since get is called using the same backward ID consecutively. maybe doesn't matter.
+        private readonly int forwardSize;
+        private readonly int backwardSize;
+        /// <summary>
+        /// Constructor for building. TODO: remove write access
+        /// </summary>
+        public ConnectionCostsWriter(int forwardSize, int backwardSize)
+        {
+            this.forwardSize = forwardSize;
+            this.backwardSize = backwardSize;
+            //this.costs = new short[backwardSize][forwardSize];
+            this.costs = Support.RectangularArrays.ReturnRectangularArray<short>(backwardSize, forwardSize);
+        }
+
+        public void Add(int forwardId, int backwardId, int cost)
+        {
+            this.costs[backwardId][forwardId] = (short)cost;
+        }
+
+        public void Write(string baseDir)
+        {
+            string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
+                typeof(ConnectionCosts).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + ConnectionCosts.FILENAME_SUFFIX;
+            //new File(filename).getParentFile().mkdirs();
+            System.IO.Directory.CreateDirectory(System.IO.Path.GetDirectoryName(filename));
+            using (Stream os = new FileStream(filename, FileMode.Create, FileAccess.Write))
+            {
+                DataOutput @out = new OutputStreamDataOutput(os);
+                CodecUtil.WriteHeader(@out, ConnectionCosts.HEADER, ConnectionCosts.VERSION);
+                @out.WriteVInt32(forwardSize);
+                @out.WriteVInt32(backwardSize);
+                int last = 0;
+                Debug.Assert(costs.Length == backwardSize);
+                foreach (short[] a in costs)
+                {
+                    Debug.Assert(a.Length == forwardSize);
+                    for (int i = 0; i < a.Length; i++)
+                    {
+                        int delta = (int)a[i] - last;
+                        @out.WriteVInt32((delta >> 31) ^ (delta << 1));
+                        last = a[i];
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs
new file mode 100644
index 0000000..c105039
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs
@@ -0,0 +1,92 @@
+using System;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class DictionaryBuilder
+    {
+        public enum DictionaryFormat { IPADIC, UNIDIC };
+
+        private DictionaryBuilder()
+        {
+        }
+
+        public static void Build(DictionaryFormat format,
+            string inputDirname,
+            string outputDirname,
+            string encoding,
+            bool normalizeEntry)
+        {
+            Console.WriteLine("building tokeninfo dict...");
+            TokenInfoDictionaryBuilder tokenInfoBuilder = new TokenInfoDictionaryBuilder(format, encoding, normalizeEntry);
+            TokenInfoDictionaryWriter tokenInfoDictionary = tokenInfoBuilder.Build(inputDirname);
+            tokenInfoDictionary.Write(outputDirname);
+            tokenInfoDictionary = null;
+            tokenInfoBuilder = null;
+            Console.WriteLine("done");
+
+            Console.WriteLine("building unknown word dict...");
+            UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
+            UnknownDictionaryWriter unkDictionary = unkBuilder.Build(inputDirname);
+            unkDictionary.Write(outputDirname);
+            unkDictionary = null;
+            unkBuilder = null;
+            Console.WriteLine("done");
+
+            Console.WriteLine("building connection costs...");
+            ConnectionCostsWriter connectionCosts
+                = ConnectionCostsBuilder.Build(inputDirname + System.IO.Path.DirectorySeparatorChar + "matrix.def");
+            connectionCosts.Write(outputDirname);
+            Console.WriteLine("done");
+        }
+
+        public static void Main(string[] args)
+        {
+            DictionaryFormat format;
+            if (args[0].Equals("ipadic", StringComparison.OrdinalIgnoreCase))
+            {
+                format = DictionaryFormat.IPADIC;
+            }
+            else if (args[0].Equals("unidic", StringComparison.OrdinalIgnoreCase))
+            {
+                format = DictionaryFormat.UNIDIC;
+            }
+            else
+            {
+                Console.Error.WriteLine("Illegal format " + args[0] + " using unidic instead");
+                format = DictionaryFormat.IPADIC;
+            }
+
+            string inputDirname = args[1];
+            string outputDirname = args[2];
+            string inputEncoding = args[3];
+            bool normalizeEntries = bool.Parse(args[4]);
+
+            Console.WriteLine("dictionary builder");
+            Console.WriteLine();
+            Console.WriteLine("dictionary format: " + format);
+            Console.WriteLine("input directory: " + inputDirname);
+            Console.WriteLine("output directory: " + outputDirname);
+            Console.WriteLine("input encoding: " + inputEncoding);
+            Console.WriteLine("normalize entries: " + normalizeEntries);
+            Console.WriteLine();
+            DictionaryBuilder.Build(format, inputDirname, outputDirname, inputEncoding, normalizeEntries);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryBuilder.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryBuilder.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryBuilder.cs
new file mode 100644
index 0000000..0406083
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryBuilder.cs
@@ -0,0 +1,230 @@
+using Lucene.Net.Support;
+using Lucene.Net.Util;
+using Lucene.Net.Util.Fst;
+using Lucene.Net.Util.Packed;
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class TokenInfoDictionaryBuilder
+    {
+        /// <summary>Internal word id - incrementally assigned as entries are read and added. This will be byte offset of dictionary file</summary>
+        private int offset = 0;
+
+        private string encoding = "euc-jp";
+
+        private bool normalizeEntries = false;
+        //private Normalizer2 normalizer;
+
+        private DictionaryBuilder.DictionaryFormat format = DictionaryBuilder.DictionaryFormat.IPADIC;
+
+        public TokenInfoDictionaryBuilder(DictionaryBuilder.DictionaryFormat format, string encoding, bool normalizeEntries)
+        {
+            this.format = format;
+            this.encoding = encoding;
+            this.normalizeEntries = normalizeEntries;
+            //this.normalizer = normalizeEntries ? Normalizer2.getInstance(null, "nfkc", Normalizer2.Mode.COMPOSE) : null;
+        }
+
+        public virtual TokenInfoDictionaryWriter Build(string dirname)
+        {
+            List<string> csvFiles = new List<string>();
+            foreach (FileInfo file in new DirectoryInfo(dirname).EnumerateFiles("*.csv"))
+            {
+                csvFiles.Add(file.FullName);
+            }
+            csvFiles.Sort();
+            return BuildDictionary(csvFiles);
+        }
+
+        public virtual TokenInfoDictionaryWriter BuildDictionary(IList<string> csvFiles)
+        {
+            TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
+
+            // all lines in the file
+            Console.WriteLine("  parse...");
+            List<string[]> lines = new List<string[]>(400000);
+            foreach (string file in csvFiles)
+            {
+                using (Stream inputStream = new FileStream(file, FileMode.Open, FileAccess.Read))
+                {
+                    Encoding decoder = Encoding.GetEncoding(encoding);
+                    TextReader reader = new StreamReader(inputStream, decoder);
+
+                    string line = null;
+                    while ((line = reader.ReadLine()) != null)
+                    {
+                        string[] entry = CSVUtil.Parse(line);
+
+                        if (entry.Length < 13)
+                        {
+                            Console.WriteLine("Entry in CSV is not valid: " + line);
+                            continue;
+                        }
+
+                        string[] formatted = FormatEntry(entry);
+                        lines.Add(formatted);
+
+                        // NFKC normalize dictionary entry
+                        if (normalizeEntries)
+                        {
+                            //if (normalizer.isNormalized(entry[0])){
+                            if (entry[0].IsNormalized(NormalizationForm.FormKC))
+                            {
+                                continue;
+                            }
+                            string[] normalizedEntry = new string[entry.Length];
+                            for (int i = 0; i < entry.Length; i++)
+                            {
+                                //normalizedEntry[i] = normalizer.normalize(entry[i]);
+                                normalizedEntry[i] = entry[i].Normalize(NormalizationForm.FormKC);
+                            }
+
+                            formatted = FormatEntry(normalizedEntry);
+                            lines.Add(formatted);
+                        }
+                    }
+                }
+            }
+
+            Console.WriteLine("  sort...");
+
+            // sort by term: we sorted the files already and use a stable sort.
+            lines.Sort(new ComparerAnonymousHelper());
+
+            Console.WriteLine("  encode...");
+
+            PositiveInt32Outputs fstOutput = PositiveInt32Outputs.Singleton;
+            Builder<long?> fstBuilder = new Builder<long?>(Lucene.Net.Util.Fst.FST.INPUT_TYPE.BYTE2, 0, 0, true, true, int.MaxValue, fstOutput, null, true, PackedInt32s.DEFAULT, true, 15);
+            Int32sRef scratch = new Int32sRef();
+            long ord = -1; // first ord will be 0
+            string lastValue = null;
+
+            // build tokeninfo dictionary
+            foreach (string[] entry in lines)
+            {
+                int next = dictionary.Put(entry);
+
+                if (next == offset)
+                {
+                    Console.WriteLine("Failed to process line: " + Collections.ToString(entry));
+                    continue;
+                }
+
+                string token = entry[0];
+                if (!token.Equals(lastValue, StringComparison.Ordinal))
+                {
+                    // new word to add to fst
+                    ord++;
+                    lastValue = token;
+                    scratch.Grow(token.Length);
+                    scratch.Length = token.Length;
+                    for (int i = 0; i < token.Length; i++)
+                    {
+                        scratch.Int32s[i] = (int)token[i];
+                    }
+                    fstBuilder.Add(scratch, ord);
+                }
+                dictionary.AddMapping((int)ord, offset);
+                offset = next;
+            }
+
+            FST<long?> fst = fstBuilder.Finish();
+
+            Console.WriteLine("  " + fst.NodeCount + " nodes, " + fst.ArcCount + " arcs, " + fst.GetSizeInBytes() + " bytes...  ");
+            dictionary.SetFST(fst);
+            Console.WriteLine(" done");
+
+            return dictionary;
+        }
+
+        private class ComparerAnonymousHelper : IComparer<string[]>
+        {
+            public int Compare(string[] left, string[] right)
+            {
+                return left[0].CompareToOrdinal(right[0]);
+            }
+        }
+
+        /// <summary>
+        /// IPADIC features
+        /// 
+        /// 0   - surface
+        /// 1   - left cost
+        /// 2   - right cost
+        /// 3   - word cost
+        /// 4-9 - pos
+        /// 10  - base form
+        /// 11  - reading
+        /// 12  - pronounciation
+        /// 
+        /// UniDic features
+        /// 
+        /// 0   - surface
+        /// 1   - left cost
+        /// 2   - right cost
+        /// 3   - word cost
+        /// 4-9 - pos
+        /// 10  - base form reading
+        /// 11  - base form
+        /// 12  - surface form
+        /// 13  - surface reading
+        /// </summary>
+        public virtual string[] FormatEntry(string[] features)
+        {
+            if (this.format == DictionaryBuilder.DictionaryFormat.IPADIC)
+            {
+                return features;
+            }
+            else
+            {
+                string[] features2 = new string[13];
+                features2[0] = features[0];
+                features2[1] = features[1];
+                features2[2] = features[2];
+                features2[3] = features[3];
+                features2[4] = features[4];
+                features2[5] = features[5];
+                features2[6] = features[6];
+                features2[7] = features[7];
+                features2[8] = features[8];
+                features2[9] = features[9];
+                features2[10] = features[11];
+
+                // If the surface reading is non-existent, use surface form for reading and pronunciation.
+                // This happens with punctuation in UniDic and there are possibly other cases as well
+                if (features[13].Length == 0)
+                {
+                    features2[11] = features[0];
+                    features2[12] = features[0];
+                }
+                else
+                {
+                    features2[11] = features[13];
+                    features2[12] = features[13];
+                }
+                return features2;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryWriter.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryWriter.cs
new file mode 100644
index 0000000..42338f9
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/TokenInfoDictionaryWriter.cs
@@ -0,0 +1,51 @@
+using Lucene.Net.Analysis.Ja.Dict;
+using Lucene.Net.Util.Fst;
+using System.IO;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class TokenInfoDictionaryWriter : BinaryDictionaryWriter
+    {
+        private FST<long?> fst;
+
+        public TokenInfoDictionaryWriter(int size)
+            : base(typeof(TokenInfoDictionary), size)
+        {
+        }
+
+        public virtual void SetFST(FST<long?> fst)
+        {
+            this.fst = fst;
+        }
+
+        public override void Write(string baseDir)
+        {
+            base.Write(baseDir);
+            WriteFST(GetBaseFileName(baseDir) + TokenInfoDictionary.FST_FILENAME_SUFFIX);
+        }
+
+        protected virtual void WriteFST(string filename)
+        {
+            FileInfo f = new FileInfo(filename);
+            if (!f.Directory.Exists) f.Directory.Create();
+            fst.Save(f);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs
new file mode 100644
index 0000000..feff4fe
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs
@@ -0,0 +1,146 @@
+using Lucene.Net.Analysis.Ja.Dict;
+using System;
+using System.Collections.Generic;
+using System.Globalization;
+using System.IO;
+using System.Text;
+using System.Text.RegularExpressions;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class UnknownDictionaryBuilder
+    {
+        private static readonly string NGRAM_DICTIONARY_ENTRY = "NGRAM,5,5,-32768,記号,一般,*,*,*,*,*,*,*";
+
+        private string encoding = "euc-jp";
+
+        public UnknownDictionaryBuilder(string encoding)
+        {
+            this.encoding = encoding;
+        }
+
+        public virtual UnknownDictionaryWriter Build(string dirname)
+        {
+            UnknownDictionaryWriter unkDictionary = ReadDictionaryFile(dirname + System.IO.Path.DirectorySeparatorChar + "unk.def");  //Should be only one file
+            ReadCharacterDefinition(dirname + System.IO.Path.DirectorySeparatorChar + "char.def", unkDictionary);
+            return unkDictionary;
+        }
+
+        public virtual UnknownDictionaryWriter ReadDictionaryFile(string filename)
+        {
+            return ReadDictionaryFile(filename, encoding);
+        }
+
+        public virtual UnknownDictionaryWriter ReadDictionaryFile(string filename, string encoding)
+        {
+            UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
+
+            List<string[]> lines = new List<string[]>();
+            Encoding decoder = Encoding.GetEncoding(encoding);
+            using (Stream inputStream = new FileStream(filename, FileMode.Open, FileAccess.Read))
+            using (TextReader reader = new StreamReader(inputStream, decoder))
+            {
+
+                dictionary.Put(CSVUtil.Parse(NGRAM_DICTIONARY_ENTRY));
+
+
+                string line = null;
+                while ((line = reader.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.
+                    string[] parsed = CSVUtil.Parse(line + ",*,*"); // Probably we don't need to validate entry
+                    lines.Add(parsed);
+                }
+            }
+
+            lines.Sort(new ComparerAnonymousHelper());
+
+            foreach (string[] entry in lines)
+            {
+                dictionary.Put(entry);
+            }
+
+            return dictionary;
+        }
+        private class ComparerAnonymousHelper : IComparer<string[]>
+        {
+            public int Compare(string[] left, string[] right)
+            {
+                int leftId = CharacterDefinition.LookupCharacterClass(left[0]);
+                int rightId = CharacterDefinition.LookupCharacterClass(right[0]);
+                return leftId - rightId;
+            }
+        }
+
+        public virtual void ReadCharacterDefinition(string filename, UnknownDictionaryWriter dictionary)
+        {
+            using (Stream inputStream = new FileStream(filename, FileMode.Open, FileAccess.Read))
+            using (TextReader reader = new StreamReader(inputStream, Encoding.GetEncoding(encoding)))
+            {
+                string line = null;
+
+                while ((line = reader.ReadLine()) != null)
+                {
+                    line = Regex.Replace(line, "^\\s", "");
+                    line = Regex.Replace(line, "\\s*#.*", "");
+                    line = Regex.Replace(line, "\\s+", " ");
+
+                    // Skip empty line or comment line
+                    if (line.Length == 0)
+                    {
+                        continue;
+                    }
+
+                    if (line.StartsWith("0x", StringComparison.Ordinal))
+                    {  // Category mapping
+                        string[] values = new Regex(" ").Split(line, 2);  // Split only first space
+
+                        if (!values[0].Contains(".."))
+                        {
+                            int cp = Convert.ToInt32(values[0]);
+                            dictionary.PutCharacterCategory(cp, values[1]);
+                        }
+                        else
+                        {
+                            string[] codePoints = Regex.Split(values[0], "\\.\\.");
+                            int cpFrom = Convert.ToInt32(codePoints[0]);
+                            int cpTo = Convert.ToInt32(codePoints[1]);
+
+                            for (int i = cpFrom; i <= cpTo; i++)
+                            {
+                                dictionary.PutCharacterCategory(i, values[1]);
+                            }
+                        }
+                    }
+                    else
+                    {  // Invoke definition
+                        string[] values = line.Split(new char[] { ' ' }, StringSplitOptions.RemoveEmptyEntries); // Consecutive space is merged above
+                        string characterClassName = values[0];
+                        int invoke = int.Parse(values[1], CultureInfo.InvariantCulture);
+                        int group = int.Parse(values[2], CultureInfo.InvariantCulture);
+                        int length = int.Parse(values[3], CultureInfo.InvariantCulture);
+                        dictionary.PutInvokeDefinition(characterClassName, invoke, group, length);
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryWriter.cs b/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryWriter.cs
new file mode 100644
index 0000000..7d4a982
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryWriter.cs
@@ -0,0 +1,66 @@
+using Lucene.Net.Analysis.Ja.Dict;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    public class UnknownDictionaryWriter : BinaryDictionaryWriter
+    {
+        private readonly CharacterDefinitionWriter characterDefinition = new CharacterDefinitionWriter();
+
+        public UnknownDictionaryWriter(int size)
+            : base(typeof(UnknownDictionary), size)
+        {
+        }
+
+        public override int Put(string[] entry)
+        {
+            // Get wordId of current entry
+            int wordId = m_buffer.Position;
+
+            // Put entry
+            int result = base.Put(entry);
+
+            // Put entry in targetMap
+            int characterId = CharacterDefinition.LookupCharacterClass(entry[0]);
+            AddMapping(characterId, wordId);
+            return result;
+        }
+
+        /// <summary>
+        /// Put mapping from unicode code point to character class.
+        /// </summary>
+        /// <param name="codePoint">Code point.</param>
+        /// <param name="characterClassName">Character class name.</param>
+        public virtual void PutCharacterCategory(int codePoint, string characterClassName)
+        {
+            characterDefinition.PutCharacterCategory(codePoint, characterClassName);
+        }
+
+        public virtual void PutInvokeDefinition(string characterClassName, int invoke, int group, int length)
+        {
+            characterDefinition.PutInvokeDefinition(characterClassName, invoke, group, length);
+        }
+
+        public override void Write(string baseDir)
+        {
+            base.Write(baseDir);
+            characterDefinition.Write(baseDir);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0f092010/src/Lucene.Net.Analysis.Kuromoji/Util/CSVUtil.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Util/CSVUtil.cs b/src/Lucene.Net.Analysis.Kuromoji/Util/CSVUtil.cs
new file mode 100644
index 0000000..9d86aed
--- /dev/null
+++ b/src/Lucene.Net.Analysis.Kuromoji/Util/CSVUtil.cs
@@ -0,0 +1,124 @@
+using System.Collections.Generic;
+using System.Text;
+using System.Text.RegularExpressions;
+
+namespace Lucene.Net.Analysis.Ja.Util
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Utility class for parsing CSV text
+    /// </summary>
+    public sealed class CSVUtil
+    {
+        private static readonly char QUOTE = '"';
+
+        private static readonly char COMMA = ',';
+
+        private static readonly Regex QUOTE_REPLACE_PATTERN = new Regex("^\"([^\"]+)\"$", RegexOptions.Compiled);
+
+        private static readonly string ESCAPED_QUOTE = "\"\"";
+
+        private CSVUtil() { } // no instance!!!
+
+        /// <summary>
+        /// Parse CSV line
+        /// </summary>
+        /// <param name="line">line containing csv-encoded data</param>
+        /// <returns>Array of values</returns>
+        public static string[] Parse(string line)
+        {
+            bool insideQuote = false;
+            List<string> result = new List<string>();
+            int quoteCount = 0;
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < line.Length; i++)
+            {
+                char c = line[i];
+
+                if (c == QUOTE)
+                {
+                    insideQuote = !insideQuote;
+                    quoteCount++;
+                }
+
+                if (c == COMMA && !insideQuote)
+                {
+                    string value = sb.ToString();
+                    value = UnQuoteUnEscape(value);
+                    result.Add(value);
+                    sb.Length = 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)
+            {
+                Match m = QUOTE_REPLACE_PATTERN.Match(original);
+                if (m.Success)
+                {
+                    result = m.Groups[1].Value;
+                }
+
+                // Unescape
+                if (result.IndexOf(ESCAPED_QUOTE) >= 0)
+                {
+                    result = result.Replace(ESCAPED_QUOTE, "\"");
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Quote and escape input value for CSV
+        /// </summary>
+        public static string QuoteEscape(string original)
+        {
+            string result = original;
+
+            if (result.IndexOf('\"') >= 0)
+            {
+                result.Replace("\"", ESCAPED_QUOTE);
+            }
+            if (result.IndexOf(COMMA) >= 0)
+            {
+                result = "\"" + result + "\"";
+            }
+            return result;
+        }
+    }
+}