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 23:28:40 UTC

[1/2] lucenenet git commit: BUG: Lucene.Net.Analysis.Kuromoji: Fixed problem with loading hexadecimal numbers from the dictionary source files

Repository: lucenenet
Updated Branches:
  refs/heads/master 2d5108ba0 -> c3f60b29f


BUG: Lucene.Net.Analysis.Kuromoji: Fixed problem with loading hexadecimal numbers from the dictionary source files


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/5c32d9e1
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/5c32d9e1
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/5c32d9e1

Branch: refs/heads/master
Commit: 5c32d9e1d8526669b296ce8cbf8616fec93086ec
Parents: 2d5108b
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Mon Jul 24 06:21:14 2017 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Mon Jul 24 06:21:14 2017 +0700

----------------------------------------------------------------------
 .../Tools/UnknownDictionaryBuilder.cs                          | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/5c32d9e1/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
index feff4fe..7f10b9b 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/UnknownDictionaryBuilder.cs
@@ -115,14 +115,14 @@ namespace Lucene.Net.Analysis.Ja.Util
 
                         if (!values[0].Contains(".."))
                         {
-                            int cp = Convert.ToInt32(values[0]);
+                            int cp = Convert.ToInt32(values[0], 16);
                             dictionary.PutCharacterCategory(cp, values[1]);
                         }
                         else
                         {
                             string[] codePoints = Regex.Split(values[0], "\\.\\.");
-                            int cpFrom = Convert.ToInt32(codePoints[0]);
-                            int cpTo = Convert.ToInt32(codePoints[1]);
+                            int cpFrom = Convert.ToInt32(codePoints[0], 16);
+                            int cpTo = Convert.ToInt32(codePoints[1], 16);
 
                             for (int i = cpFrom; i <= cpTo; i++)
                             {


[2/2] lucenenet git commit: ENHANCEMENT: Lucene.Net.Analysis.Kuromoji: Changed the way the files are loaded to be similar to the SmartCn package so the dictionary files can be customized without having to recompile the library. Added a smoke test to catc

Posted by ni...@apache.org.
ENHANCEMENT: Lucene.Net.Analysis.Kuromoji: Changed the way the files are loaded to be similar to the SmartCn package so the dictionary files can be customized without having to recompile the library. Added a smoke test to catch any exceptions thrown by the DictionaryBuilder.Main() command. Added code to load the extended set of codepages (including Japanese) on .NET Standard and changed the default encoding to EUC-JP.


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/c3f60b29
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/c3f60b29
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/c3f60b29

Branch: refs/heads/master
Commit: c3f60b29f54ac1c5cc2d3e94f17a27208c13683c
Parents: 5c32d9e
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Mon Jul 24 06:27:27 2017 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Mon Jul 24 06:27:27 2017 +0700

----------------------------------------------------------------------
 .../Dict/BinaryDictionary.cs                    |  89 ++++++++++++++++++-
 .../JapaneseTokenizerFactory.cs                 |   9 ++
 .../Tools/BinaryDictionaryWriter.cs             |  13 ++-
 .../Tools/CharacterDefinitionWriter.cs          |   9 +-
 .../Tools/ConnectionCostsWriter.cs              |   9 +-
 .../Tools/DictionaryBuilder.cs                  |   9 ++
 .../Lucene.Net.Tests.Analysis.Kuromoji.csproj   |   2 +
 .../Tools/TestBuildDictionary.cs                |  59 ++++++++++++
 .../Tools/custom-dictionary-input.zip           | Bin 0 -> 5279887 bytes
 .../project.json                                |   3 +-
 ...nalysisKuromojiBuildDictionaryCommandTest.cs |   4 +-
 .../lucene-cli/Resources/Strings.Designer.cs    |  25 +++++-
 src/tools/lucene-cli/Resources/Strings.resx     |  21 ++++-
 .../AnalysisKuromojiBuildDictionaryCommand.cs   |   3 +-
 .../docs/analysis/kuromoji-build-dictionary.md  |  22 ++++-
 15 files changed, 254 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Analysis.Kuromoji/Dict/BinaryDictionary.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/Dict/BinaryDictionary.cs b/src/Lucene.Net.Analysis.Kuromoji/Dict/BinaryDictionary.cs
index 4068e38..86b3598 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Dict/BinaryDictionary.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Dict/BinaryDictionary.cs
@@ -6,6 +6,7 @@ using Lucene.Net.Util;
 using System;
 using System.IO;
 using System.Reflection;
+using System.Security;
 
 namespace Lucene.Net.Analysis.Ja.Dict
 {
@@ -46,6 +47,50 @@ namespace Lucene.Net.Analysis.Ja.Dict
         private readonly string[] inflTypeDict;
         private readonly string[] inflFormDict;
 
+        // LUCENENET specific - variable to hold the name of the data directory (or empty string to load embedded resources)
+        private static readonly string DATA_DIR;
+        // LUCENENET specific - name of the subdirectory inside of the directory where the Kuromoji dictionary files reside.
+        private static readonly string DATA_SUBDIR = "kuromoji-data";
+
+        static BinaryDictionary()
+        {
+            string currentPath = GetSystemProperty("kuromoji.data.dir",
+#if NETSTANDARD
+                System.AppContext.BaseDirectory
+#else
+                AppDomain.CurrentDomain.BaseDirectory
+#endif
+                );
+
+            // If a matching directory path is found, set our DATA_DIR static
+            // variable. If it is null or empty after this process, we need to
+            // load the embedded files.
+            string candidatePath = System.IO.Path.Combine(currentPath, DATA_SUBDIR);
+            if (System.IO.Directory.Exists(candidatePath))
+            {
+                DATA_DIR = candidatePath;
+                return;
+            }
+
+            while (new DirectoryInfo(currentPath).Parent != null)
+            {
+                try
+                {
+                    candidatePath = System.IO.Path.Combine(new DirectoryInfo(currentPath).Parent.FullName, DATA_SUBDIR);
+                    if (System.IO.Directory.Exists(candidatePath))
+                    {
+                        DATA_DIR = candidatePath;
+                        return;
+                    }
+                    currentPath = new DirectoryInfo(currentPath).Parent.FullName;
+                }
+                catch (SecurityException)
+                {
+                    // ignore security errors
+                }
+            }
+        }
+
         protected BinaryDictionary()
         {
             int[] targetMapOffsets = null, targetMap = null;
@@ -135,10 +180,46 @@ namespace Lucene.Net.Analysis.Ja.Dict
         // util, reused by ConnectionCosts and CharacterDefinition
         public static Stream GetTypeResource(Type clazz, string suffix)
         {
-            Stream @is = clazz.GetTypeInfo().Assembly.FindAndGetManifestResourceStream(clazz, clazz.Name + suffix);
-            if (@is == null)
-                throw new FileNotFoundException("Not in assembly: " + clazz.FullName + suffix);
-            return @is;
+            string fileName = clazz.Name + suffix;
+
+            // LUCENENET specific: Rather than forcing the end user to recompile if they want to use a custom dictionary,
+            // we load the data from the kuromoji-data directory (which can be set via the kuromoji.data.dir environment variable).
+            if (string.IsNullOrEmpty(DATA_DIR))
+            {
+                Stream @is = clazz.GetTypeInfo().Assembly.FindAndGetManifestResourceStream(clazz, fileName);
+                if (@is == null)
+                    throw new FileNotFoundException("Not in assembly: " + clazz.FullName + suffix);
+                return @is;
+            }
+
+            // We have a data directory, so first check if the file exists
+            string path = System.IO.Path.Combine(DATA_DIR, fileName);
+            if (!System.IO.File.Exists(path))
+            {
+                throw new FileNotFoundException(string.Format("Expected file '{0}' not found. " +
+                    "If the '{1}' directory exists, this file is required. " +
+                    "Either remove the '{3}' directory or generate the required dictionary files using the lucene-cli tool.",
+                    fileName, DATA_DIR, DATA_SUBDIR));
+            }
+
+            // The file exists - open a stream.
+            return new FileStream(path, FileMode.Open, FileAccess.Read);
+        }
+
+        // LUCENENET specific helper to load environment variables and ignore security exceptions
+        private static string GetSystemProperty(string property, string defaultValue)
+        {
+            string setting;
+            try
+            {
+                setting = Environment.GetEnvironmentVariable(property);
+            }
+            catch (SecurityException)
+            {
+                setting = null;
+            }
+
+            return (setting == null) ? defaultValue : setting;
         }
 
         public virtual void LookupWordIds(int sourceId, Int32sRef @ref)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Analysis.Kuromoji/JapaneseTokenizerFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Kuromoji/JapaneseTokenizerFactory.cs b/src/Lucene.Net.Analysis.Kuromoji/JapaneseTokenizerFactory.cs
index 5fe99d5..4d9b222 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/JapaneseTokenizerFactory.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/JapaneseTokenizerFactory.cs
@@ -72,6 +72,15 @@ namespace Lucene.Net.Analysis.Ja
             }
         }
 
+        static JapaneseTokenizerFactory()
+        {
+#if NETSTANDARD
+            // Support for EUC-JP encoding. See: https://docs.microsoft.com/en-us/dotnet/api/system.text.codepagesencodingprovider?view=netcore-2.0
+            var encodingProvider = System.Text.CodePagesEncodingProvider.Instance;
+            System.Text.Encoding.RegisterProvider(encodingProvider);
+#endif
+        }
+
         public virtual void Inform(IResourceLoader loader)
         {
             if (userDictionaryPath != null)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/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
index 39d36eb..c83b13d 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/BinaryDictionaryWriter.cs
@@ -263,7 +263,12 @@ namespace Lucene.Net.Analysis.Ja.Util
 
         protected string GetBaseFileName(string baseDir)
         {
-            return baseDir + System.IO.Path.DirectorySeparatorChar + m_implClazz.FullName.Replace('.', System.IO.Path.DirectorySeparatorChar);
+            // LUCENENET specific: we don't need to do a "classpath" output directory, since we
+            // are changing the implementation to read files dynamically instead of making the
+            // user recompile with the new files.
+            return System.IO.Path.Combine(baseDir, m_implClazz.Name);
+
+            //return baseDir + System.IO.Path.DirectorySeparatorChar + m_implClazz.FullName.Replace('.', System.IO.Path.DirectorySeparatorChar);
         }
 
         /// <summary>
@@ -354,14 +359,16 @@ namespace Lucene.Net.Analysis.Ja.Util
                 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);
+                while (m_buffer.HasRemaining)
+                {
+                    @out.WriteByte(m_buffer.Get());
+                }
 
                 Debug.Assert(m_buffer.Remaining == 0L);
             }

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/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
index 6d5526d..39ed4f8 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/CharacterDefinitionWriter.cs
@@ -68,8 +68,13 @@ namespace Lucene.Net.Analysis.Ja.Util
 
         public void Write(string baseDir)
         {
-            string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
-                typeof(CharacterDefinition).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + CharacterDefinition.FILENAME_SUFFIX;
+            //string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
+            //    typeof(CharacterDefinition).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + CharacterDefinition.FILENAME_SUFFIX;
+
+            // LUCENENET specific: we don't need to do a "classpath" output directory, since we
+            // are changing the implementation to read files dynamically instead of making the
+            // user recompile with the new files.
+            string filename = System.IO.Path.Combine(baseDir, typeof(CharacterDefinition).Name + 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))

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/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
index bd1376e..91e4df8 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/ConnectionCostsWriter.cs
@@ -46,8 +46,13 @@ namespace Lucene.Net.Analysis.Ja.Util
 
         public void Write(string baseDir)
         {
-            string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
-                typeof(ConnectionCosts).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + ConnectionCosts.FILENAME_SUFFIX;
+            //string filename = baseDir + System.IO.Path.DirectorySeparatorChar +
+            //    typeof(ConnectionCosts).FullName.Replace('.', System.IO.Path.DirectorySeparatorChar) + ConnectionCosts.FILENAME_SUFFIX;
+
+            // LUCENENET specific: we don't need to do a "classpath" output directory, since we
+            // are changing the implementation to read files dynamically instead of making the
+            // user recompile with the new files.
+            string filename = System.IO.Path.Combine(baseDir, typeof(ConnectionCosts).Name + CharacterDefinition.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))

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/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
index c105039..07e92f7 100644
--- a/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs
+++ b/src/Lucene.Net.Analysis.Kuromoji/Tools/DictionaryBuilder.cs
@@ -27,6 +27,15 @@ namespace Lucene.Net.Analysis.Ja.Util
         {
         }
 
+        static DictionaryBuilder()
+        {
+#if NETSTANDARD
+            // Support for EUC-JP encoding. See: https://docs.microsoft.com/en-us/dotnet/api/system.text.codepagesencodingprovider?view=netcore-2.0
+            var encodingProvider = System.Text.CodePagesEncodingProvider.Instance;
+            System.Text.Encoding.RegisterProvider(encodingProvider);
+#endif
+        }
+
         public static void Build(DictionaryFormat format,
             string inputDirname,
             string outputDirname,

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Tests.Analysis.Kuromoji/Lucene.Net.Tests.Analysis.Kuromoji.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Kuromoji/Lucene.Net.Tests.Analysis.Kuromoji.csproj b/src/Lucene.Net.Tests.Analysis.Kuromoji/Lucene.Net.Tests.Analysis.Kuromoji.csproj
index e2cbbb8..8fd2942 100644
--- a/src/Lucene.Net.Tests.Analysis.Kuromoji/Lucene.Net.Tests.Analysis.Kuromoji.csproj
+++ b/src/Lucene.Net.Tests.Analysis.Kuromoji/Lucene.Net.Tests.Analysis.Kuromoji.csproj
@@ -60,6 +60,7 @@
     <Compile Include="TestJapaneseTokenizer.cs" />
     <Compile Include="TestJapaneseTokenizerFactory.cs" />
     <Compile Include="TestSearchMode.cs" />
+    <Compile Include="Tools\TestBuildDictionary.cs" />
     <Compile Include="Tools\UnknownDictionaryTest.cs" />
     <Compile Include="Util\TestToStringUtil.cs" />
     <Compile Include="..\CommonAssemblyInfo.cs">
@@ -87,6 +88,7 @@
   <ItemGroup>
     <EmbeddedResource Include="bocchan.utf-8" />
     <None Include="Lucene.Net.Tests.Analysis.Kuromoji.project.json" />
+    <EmbeddedResource Include="Tools\custom-dictionary-input.zip" />
   </ItemGroup>
   <ItemGroup>
     <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/TestBuildDictionary.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/TestBuildDictionary.cs b/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/TestBuildDictionary.cs
new file mode 100644
index 0000000..9f13314
--- /dev/null
+++ b/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/TestBuildDictionary.cs
@@ -0,0 +1,59 @@
+using Lucene.Net.Analysis.Ja.Util;
+using Lucene.Net.Attributes;
+using Lucene.Net.Support;
+using Lucene.Net.Util;
+using NUnit.Framework;
+using System.Collections.Generic;
+using System.Reflection;
+
+namespace Lucene.Net.Analysis.Ja.Tools
+{
+    /*
+     * 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 TestBuildDictionary : LuceneTestCase
+    {
+        /// <summary>
+        /// Since there were no tests provided for the BuildDictionary tool, this is a smoke test
+        /// just to ensure the I/O has basic functionality.
+        /// <para/>
+        /// The data was sourced from:
+        /// http://mentaldetritus.blogspot.com/2013/03/compiling-custom-dictionary-for.html
+        /// https://sourceforge.net/projects/mecab/files/mecab-ipadic/2.7.0-20070801/
+        /// </summary>
+        [Test]
+        [LuceneNetSpecific]
+        public void TestBuildDictionaryEndToEnd()
+        {
+            var inputDir = CreateTempDir("build-dictionary-input");
+            var outputDir = CreateTempDir("build-dictionary-output");
+            using (var zipFileStream = this.GetType().GetTypeInfo().Assembly.FindAndGetManifestResourceStream(this.GetType(), "custom-dictionary-input.zip"))
+            {
+                TestUtil.Unzip(zipFileStream, inputDir);
+            }
+
+            var args = new List<string>();
+            args.Add("ipadic"); // dictionary format
+            args.Add(inputDir.FullName); // input dir
+            args.Add(outputDir.FullName); // output dir
+            args.Add("euc-jp"); // encoding
+            args.Add("true"); // normalize?
+
+            DictionaryBuilder.Main(args.ToArray());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/custom-dictionary-input.zip
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/custom-dictionary-input.zip b/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/custom-dictionary-input.zip
new file mode 100644
index 0000000..51fd797
Binary files /dev/null and b/src/Lucene.Net.Tests.Analysis.Kuromoji/Tools/custom-dictionary-input.zip differ

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/Lucene.Net.Tests.Analysis.Kuromoji/project.json
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Tests.Analysis.Kuromoji/project.json b/src/Lucene.Net.Tests.Analysis.Kuromoji/project.json
index 5badefa..55102b8 100644
--- a/src/Lucene.Net.Tests.Analysis.Kuromoji/project.json
+++ b/src/Lucene.Net.Tests.Analysis.Kuromoji/project.json
@@ -9,7 +9,8 @@
       "includeFiles": [
         "bocchan.utf-8",
         "search-segmentation-tests.txt",
-        "userdict.txt"
+        "userdict.txt",
+        "Tools/custom-dictionary-input.zip"
       ]
     }
   },

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/tools/Lucene.Net.Tests.Cli/Commands/Analysis/AnalysisKuromojiBuildDictionaryCommandTest.cs
----------------------------------------------------------------------
diff --git a/src/tools/Lucene.Net.Tests.Cli/Commands/Analysis/AnalysisKuromojiBuildDictionaryCommandTest.cs b/src/tools/Lucene.Net.Tests.Cli/Commands/Analysis/AnalysisKuromojiBuildDictionaryCommandTest.cs
index c8eaa41..dd77eea 100644
--- a/src/tools/Lucene.Net.Tests.Cli/Commands/Analysis/AnalysisKuromojiBuildDictionaryCommandTest.cs
+++ b/src/tools/Lucene.Net.Tests.Cli/Commands/Analysis/AnalysisKuromojiBuildDictionaryCommandTest.cs
@@ -66,7 +66,7 @@ namespace Lucene.Net.Cli.Commands
                     
                     .Concat(new string[] {
                         // Special case: the encoding must always be supplied
-                        "utf-8",
+                        "euc-jp",
                         // Special case: normalize must always be supplied
                         "false"
                     }).ToArray());
@@ -79,7 +79,7 @@ namespace Lucene.Net.Cli.Commands
                     string command = string.Join(" ", requiredArg.Select(x => x.InputPattern).Union(optionalArg.Select(x => x.InputPattern).ToArray()));
                     string[] expected = requiredArg.SelectMany(x => x.Output)
                         // Special case: the encoding must always be supplied
-                        .Concat(Regex.IsMatch(command, "-e|--encoding") ? new string[] { "UTF-16" } : new string[] { "utf-8" })
+                        .Concat(Regex.IsMatch(command, "-e|--encoding") ? new string[] { "UTF-16" } : new string[] { "euc-jp" })
                         // Special case: the encoding must always be supplied
                         .Concat(Regex.IsMatch(command, "-n|--normalize") ? new string[] { "true" } : new string[] { "false" }).ToArray();
                     AssertCommandTranslation(command, expected);

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/tools/lucene-cli/Resources/Strings.Designer.cs
----------------------------------------------------------------------
diff --git a/src/tools/lucene-cli/Resources/Strings.Designer.cs b/src/tools/lucene-cli/Resources/Strings.Designer.cs
index 9af44ff..6cbbdde 100644
--- a/src/tools/lucene-cli/Resources/Strings.Designer.cs
+++ b/src/tools/lucene-cli/Resources/Strings.Designer.cs
@@ -69,7 +69,7 @@ namespace Lucene.Net.Cli.Resources {
         }
         
         /// <summary>
-        ///    Looks up a localized string similar to Builds a custom dictionary that can be used by the JapaneseAnalyzer or JapaneseTokenizer..
+        ///    Looks up a localized string similar to Generates a set of custom dictionary files for the Lucene.Net.Analysis.Kuromoji library..
         /// </summary>
         public static string AnalysisKuromojiBuildDictionaryCommandDescription {
             get {
@@ -78,6 +78,27 @@ namespace Lucene.Net.Cli.Resources {
         }
         
         /// <summary>
+        ///    Looks up a localized string similar to  Generates the following set of binary files:
+        ///
+        ///- CharacterDefinition.dat
+        ///- ConnectionCosts.dat
+        ///- TokenInfoDictionary$buffer.dat
+        ///- TokenInfoDictionary$fst.dat
+        ///- TokenInfoDictionary$posDict.dat
+        ///- TokenInfoDictionary$targetMap.dat
+        ///- UnknownDictionary$buffer.dat
+        ///- UnknownDictionary$posDict.dat
+        ///- UnknownDictionary$targetMap.dat
+        ///
+        ///If these files are placed into a subdirectory of your application named &apos;kuromoji-data&apos;, they will be used automatically by Lucene.Net.Analysis.Kuromoji features such as the J [rest of string was truncated]&quot;;.
+        /// </summary>
+        public static string AnalysisKuromojiBuildDictionaryCommandExtendedHelpText {
+            get {
+                return ResourceManager.GetString("AnalysisKuromojiBuildDictionaryCommandExtendedHelpText", resourceCulture);
+            }
+        }
+        
+        /// <summary>
         ///    Looks up a localized string similar to The dictionary format. Valid values are IPADIC and UNIDIC. If an invalid value is passed, IPADIC is assumed..
         /// </summary>
         public static string AnalysisKuromojiBuildDictionaryCommandFormatDescription {
@@ -96,7 +117,7 @@ namespace Lucene.Net.Cli.Resources {
         }
         
         /// <summary>
-        ///    Looks up a localized string similar to The file encoding used by the input files. If not supplied, the default value is `UTF-8`..
+        ///    Looks up a localized string similar to The file encoding used by the input files. If not supplied, the default value is EUC-JP..
         /// </summary>
         public static string AnalysisKuromojiBuildDictionaryCommandInputDirectoryEncodingDescription {
             get {

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/tools/lucene-cli/Resources/Strings.resx
----------------------------------------------------------------------
diff --git a/src/tools/lucene-cli/Resources/Strings.resx b/src/tools/lucene-cli/Resources/Strings.resx
index 727cb62..4b1a488 100644
--- a/src/tools/lucene-cli/Resources/Strings.resx
+++ b/src/tools/lucene-cli/Resources/Strings.resx
@@ -121,7 +121,24 @@
     <value>Utilities to manage specialized analyzers.</value>
   </data>
   <data name="AnalysisKuromojiBuildDictionaryCommandDescription" xml:space="preserve">
-    <value>Builds a custom dictionary that can be used by the JapaneseAnalyzer or JapaneseTokenizer.</value>
+    <value>Generates a set of custom dictionary files for the Lucene.Net.Analysis.Kuromoji library.</value>
+  </data>
+  <data name="AnalysisKuromojiBuildDictionaryCommandExtendedHelpText" xml:space="preserve">
+    <value>Generates the following set of binary files:
+
+- CharacterDefinition.dat
+- ConnectionCosts.dat
+- TokenInfoDictionary$buffer.dat
+- TokenInfoDictionary$fst.dat
+- TokenInfoDictionary$posDict.dat
+- TokenInfoDictionary$targetMap.dat
+- UnknownDictionary$buffer.dat
+- UnknownDictionary$posDict.dat
+- UnknownDictionary$targetMap.dat
+
+If these files are placed into a subdirectory of your application named 'kuromoji-data', they will be used automatically by Lucene.Net.Analysis.Kuromoji features such as the JapaneseAnalyzer or JapaneseTokenizer. To use an alternate directory location, put the path in an environment variable named 'kuromoji.data.dir'. The files must be placed in a subdirectory of this location named 'kuromoji-data'.
+
+See this blog post (http://mentaldetritus.blogspot.com/2013/03/compiling-custom-dictionary-for.html) for information about the dictionary format. A sample is available at (https://sourceforge.net/projects/mecab/files/mecab-ipadic/2.7.0-20070801/). The Kuromoji project documentation (https://github.com/atilika/kuromoji) may also be helpful.</value>
   </data>
   <data name="AnalysisKuromojiBuildDictionaryCommandFormatDescription" xml:space="preserve">
     <value>The dictionary format. Valid values are IPADIC and UNIDIC. If an invalid value is passed, IPADIC is assumed.</value>
@@ -130,7 +147,7 @@
     <value>The directory where the dictionary input files are located.</value>
   </data>
   <data name="AnalysisKuromojiBuildDictionaryCommandInputDirectoryEncodingDescription" xml:space="preserve">
-    <value>The file encoding used by the input files. If not supplied, the default value is `UTF-8`.</value>
+    <value>The file encoding used by the input files. If not supplied, the default value is EUC-JP.</value>
   </data>
   <data name="AnalysisKuromojiBuildDictionaryCommandNormalizeDescription" xml:space="preserve">
     <value>Normalize the entries using normalization form KC.</value>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/tools/lucene-cli/commands/analysis/analysis-kuromoji-build-dictionary/AnalysisKuromojiBuildDictionaryCommand.cs
----------------------------------------------------------------------
diff --git a/src/tools/lucene-cli/commands/analysis/analysis-kuromoji-build-dictionary/AnalysisKuromojiBuildDictionaryCommand.cs b/src/tools/lucene-cli/commands/analysis/analysis-kuromoji-build-dictionary/AnalysisKuromojiBuildDictionaryCommand.cs
index 7f10ed7..5232726 100644
--- a/src/tools/lucene-cli/commands/analysis/analysis-kuromoji-build-dictionary/AnalysisKuromojiBuildDictionaryCommand.cs
+++ b/src/tools/lucene-cli/commands/analysis/analysis-kuromoji-build-dictionary/AnalysisKuromojiBuildDictionaryCommand.cs
@@ -31,6 +31,7 @@ namespace Lucene.Net.Cli
 
                 this.Name = "kuromoji-build-dictionary";
                 this.Description = FromResource("Description");
+                this.ExtendedHelpText = FromResource("ExtendedHelpText");
 
                 this.Format = this.Argument(
                     "<FORMAT>",
@@ -76,7 +77,7 @@ namespace Lucene.Net.Cli
             }
             else
             {
-                args.Add("utf-8");
+                args.Add("euc-jp");
             }
 
             if (input.Normalize.HasValue())

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/c3f60b29/src/tools/lucene-cli/docs/analysis/kuromoji-build-dictionary.md
----------------------------------------------------------------------
diff --git a/src/tools/lucene-cli/docs/analysis/kuromoji-build-dictionary.md b/src/tools/lucene-cli/docs/analysis/kuromoji-build-dictionary.md
index 9fd7cf6..d74b53c 100644
--- a/src/tools/lucene-cli/docs/analysis/kuromoji-build-dictionary.md
+++ b/src/tools/lucene-cli/docs/analysis/kuromoji-build-dictionary.md
@@ -2,7 +2,7 @@
 
 ### Name
 
-`analysis-kuromoji-build-dictionary` - Generates a dictionary file for the JapaneseAnalyzer or JapaneseTokenizer in the Lucene.Net.Analysis.Kuromoji project.
+`analysis-kuromoji-build-dictionary` - Generates a set of custom dictionary files for the Lucene.Net.Analysis.Kuromoji library.
 
 ### Synopsis
 
@@ -10,7 +10,21 @@
 
 ### Description
 
-See the [Kuromoji project documentation](https://github.com/atilika/kuromoji) for more information.
+Generates the following set of binary files:
+
+- CharacterDefinition.dat
+- ConnectionCosts.dat
+- TokenInfoDictionary$buffer.dat
+- TokenInfoDictionary$fst.dat
+- TokenInfoDictionary$posDict.dat
+- TokenInfoDictionary$targetMap.dat
+- UnknownDictionary$buffer.dat
+- UnknownDictionary$posDict.dat
+- UnknownDictionary$targetMap.dat
+
+If these files are placed into a subdirectory of your application named `kuromoji-data`, they will be used automatically by Lucene.Net.Analysis.Kuromoji features such as the JapaneseAnalyzer or JapaneseTokenizer. To use an alternate directory location, put the path in an environment variable named `kuromoji.data.dir`. The files must be placed in a subdirectory of this location named `kuromoji-data`.
+
+See [this blog post](http://mentaldetritus.blogspot.com/2013/03/compiling-custom-dictionary-for.html) for information about the dictionary format. A sample is available at [https://sourceforge.net/projects/mecab/files/mecab-ipadic/2.7.0-20070801/](https://sourceforge.net/projects/mecab/files/mecab-ipadic/2.7.0-20070801/). The [Kuromoji project documentation](https://github.com/atilika/kuromoji) may also be helpful. 
 
 ### Arguments
 
@@ -34,7 +48,7 @@ Prints out a short help for the command.
 
 `-e|--encoding <ENCODING>`
 
-The file encoding used by the input files. If not supplied, the default value is `UTF-8`.
+The file encoding used by the input files. If not supplied, the default value is `EUC-JP`.
 
 `-n|--normalize`
 
@@ -42,5 +56,5 @@ Normalize the entries using normalization form KC.
 
 ### Example
 
-<code>dotnet lucene-cli.dll analysis kuromoji-build-dictionary X:\kuromoji-data X:\kuromoji-dictionary --encoding UTF-16</code>
+<code>dotnet lucene-cli.dll analysis kuromoji-build-dictionary X:\kuromoji-data X:\kuromoji-dictionary --normalize</code>