You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/05 15:58:51 UTC

svn commit: r1227618 - in /lucene/dev/branches/lucene3305/modules/analysis/kuromoji: ./ src/java/org/apache/lucene/analysis/kuromoji/dict/ src/test/org/apache/lucene/analysis/kuromoji/dict/ src/tools/java/org/apache/lucene/analysis/kuromoji/util/ src/t...

Author: rmuir
Date: Thu Jan  5 14:58:50 2012
New Revision: 1227618

URL: http://svn.apache.org/viewvc?rev=1227618&view=rev
Log:
LUCENE-3305: split dictionary reading/writing, nuke files before build-dict to ensure nothing is stale, ensure tools/ breakage breaks the build since its important here

Added:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java   (with props)
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java   (with props)
Removed:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java
Modified:
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/build.xml
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
    lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/build.xml?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/build.xml (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/build.xml Thu Jan  5 14:58:50 2012
@@ -54,34 +54,68 @@
     <fileset dir="../icu/lib" includes="icu4j-*.jar"/>
   </path>
 
+  <path id="tools.classpath">
+    <path refid="classpath"/>
+    <path refid="tools.dependencies"/>
+    <pathelement location="${build.dir}/classes/java"/>
+    <pathelement location="${build.dir}/classes/tools"/>
+  </path>
+
+  <path id="tools.test.classpath">
+    <path refid="tools.classpath"/>
+    <path refid="test.base.classpath"/>
+    <pathelement location="${build.dir}/classes/tools-test"/>
+  </path>
+
   <target name="build-dict" depends="compile-tools, download-dict">
-    <java fork="true" failonerror="true" maxmemory="512m" classname="org.apache.lucene.analysis.kuromoji.util.DictionaryBuilder">
-      <classpath>
-        <path refid="classpath"/>
-        <pathelement path="${build.dir}/classes/java"/>
-        <pathelement path="${build.dir}/classes/tools"/>
-        <path refid="tools.dependencies"/>
-      </classpath>
-      <assertions>
-        <enable package="org.apache.lucene"/>
-      </assertions>
-      <arg value="${dict.format}"/>
-      <arg value="${dict.src.dir}"/>
-      <arg value="${dict.target.dir}"/>
-      <arg value="${dict.encoding}"/>
-      <arg value="${dict.normalize}"/>
-    </java>
+    <sequential>
+      <delete verbose="true">
+        <fileset dir="src/resources/org/apache/lucene/analysis/kuromoji" includes="**/*"/>
+      </delete>
+      <java fork="true" failonerror="true" maxmemory="512m" classname="org.apache.lucene.analysis.kuromoji.util.DictionaryBuilder">
+        <classpath>
+          <path refid="tools.classpath"/>
+          <pathelement path="${build.dir}/classes/tools"/>
+        </classpath>
+        <assertions>
+          <enable package="org.apache.lucene"/>
+        </assertions>
+        <arg value="${dict.format}"/>
+        <arg value="${dict.src.dir}"/>
+        <arg value="${dict.target.dir}"/>
+        <arg value="${dict.encoding}"/>
+        <arg value="${dict.normalize}"/>
+      </java>
+    </sequential>
   </target>
 
-    <target name="compile-tools" depends="compile-core, common.compile-tools">
+  <target name="compile-tools" depends="compile-core, common.compile-tools">
     <compile
       srcdir="src/tools/java"
       destdir="${build.dir}/classes/tools">
       <classpath>
-        <path refid="classpath"/>
-        <pathelement path="${build.dir}/classes/java"/>
-        <path refid="tools.dependencies"/>
+        <path refid="tools.classpath"/>
+        <pathelement path="src/tools/java"/>
       </classpath>
     </compile>
   </target>
+
+  <target name="compile-tools-tests" depends="compile-tools">
+    <compile
+      srcdir="src/tools/test"
+      destdir="${build.dir}/classes/tools-test">
+      <classpath>
+        <path refid="tools.test.classpath"/>
+        <pathelement path="src/tools/test"/>
+      </classpath>
+     </compile>
+  </target>
+
+  <target name="test-tools" depends="compile-tools-tests">
+    <test-macro dataDir="src/tools/test" junit.classpath="tools.test.classpath"/>
+  </target>
+
+  <target name="compile-test" depends="contrib-build.compile-test, compile-tools-tests"/>
+  <target name="test" depends="contrib-build.test, test-tools"/>
+
 </project>

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/TokenInfoDictionary.java Thu Jan  5 14:58:50 2012
@@ -18,24 +18,15 @@ package org.apache.lucene.analysis.kurom
  */
 
 import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
 
 import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.CodecUtil;
 
 import org.apache.lucene.analysis.kuromoji.util.CSVUtil;
@@ -53,72 +44,6 @@ public class TokenInfoDictionary impleme
   
   protected int[][] targetMap;
   
-  public TokenInfoDictionary() {
-  }
-  
-  public TokenInfoDictionary(int size) {
-    targetMap = new int[1][];
-    buffer = ByteBuffer.allocate(size);
-  }
-  
-  /**
-   * put the entry in map
-   * @return current position of buffer, which will be wordId of next entry
-   */
-  public int put(String[] entry) {
-    short leftId = Short.parseShort(entry[1]);
-    short rightId = Short.parseShort(entry[2]);
-    short wordCost = Short.parseShort(entry[3]);
-    
-    StringBuilder sb = new StringBuilder();
-    for (int i = 4; i < entry.length; i++){
-      sb.append(entry[i]).append(INTERNAL_SEPARATOR);
-    }
-    String features = sb.deleteCharAt(sb.length() - 1).toString();
-    int featuresSize = features.length()* 2;
-    
-    // extend buffer if necessary
-    int left = buffer.limit() - buffer.position();
-    if (8 + featuresSize > left) { // four short and features
-      ByteBuffer newBuffer = ByteBuffer.allocate(buffer.limit() * 2);
-      buffer.flip();
-      newBuffer.put(buffer);
-      buffer = newBuffer;
-    }
-    
-    buffer.putShort(leftId);
-    buffer.putShort(rightId);
-    buffer.putShort(wordCost);
-    buffer.putShort((short)featuresSize);
-    for (char c : features.toCharArray()){
-      buffer.putChar(c);
-    }
-    
-    return buffer.position();
-  }
-  
-  public void addMapping(int sourceId, int wordId) {
-    if(targetMap.length <= sourceId) {
-      int[][] newArray = new int[sourceId + 1][];
-      System.arraycopy(targetMap, 0, newArray, 0, targetMap.length);
-      targetMap = newArray;
-    }
-    
-    // Prepare array -- extend the length of array by one
-    int[] current = targetMap[sourceId];
-    if (current == null) {
-      current = new int[1];
-    } else {
-      int[] newArray = new int[current.length + 1];
-      System.arraycopy(current, 0, newArray, 0, current.length);
-      current = newArray;
-    }
-    targetMap[sourceId] = current;
-    
-    int[] targets = targetMap[sourceId];
-    targets[targets.length - 1] = wordId;
-  }
-  
   public int[] lookupWordIds(int sourceId) {
     return targetMap[sourceId];
   }
@@ -190,69 +115,6 @@ public class TokenInfoDictionary impleme
     String form = getFeature(wordId, 6);
     return "*".equals(form) ? null : form;
   }
-
-  /**
-   * Write dictionary in file
-   * 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...].....
-   * @throws IOException
-   */
-  public void write(String directoryname) throws IOException {
-    writeDictionary(directoryname + File.separator + FILENAME);
-    writeTargetMap(directoryname + File.separator + TARGETMAP_FILENAME);
-  }
-  
-  protected void writeTargetMap(String filename) throws IOException {
-    OutputStream os = new FileOutputStream(filename);
-    try {
-      os = new BufferedOutputStream(os);
-      final DataOutput out = new OutputStreamDataOutput(os);
-      CodecUtil.writeHeader(out, TARGETMAP_HEADER, VERSION);
-      out.writeVInt(targetMap.length);
-      int nulls = 0;
-      for (int[] a : targetMap) {
-        if (a == null) {
-          // run-length encoding for all nulls:
-          if (nulls == 0) {
-            out.writeVInt(0);
-          }
-          nulls++;
-        } else {
-          if (nulls > 0) {
-            out.writeVInt(nulls);
-            nulls = 0;
-          }
-          assert a.length > 0;
-          out.writeVInt(a.length);
-          for (int i = 0; i < a.length; i++) {
-            out.writeVInt(a[i]);
-          }
-        }
-      }
-      // write the pending RLE count:
-      if (nulls > 0) {
-        out.writeVInt(nulls);
-      }
-    } finally {
-      os.close();
-    }
-  }
-  
-  protected void writeDictionary(String filename) throws IOException {
-    final FileOutputStream os = new FileOutputStream(filename);
-    try {
-      final DataOutput out = new OutputStreamDataOutput(os);
-      CodecUtil.writeHeader(out, DICT_HEADER, VERSION);
-      out.writeVInt(buffer.position());
-      final WritableByteChannel channel = Channels.newChannel(os);
-      // Write Buffer
-      buffer.flip();  // set position to 0, set limit to current position
-      channel.write(buffer);
-      assert buffer.remaining() == 0L;
-    } finally {
-      os.close();
-    }
-  }
   
   /**
    * Read dictionary into directly allocated buffer.
@@ -308,5 +170,4 @@ public class TokenInfoDictionary impleme
       is.close();
     }
   }
-  
 }

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary.java?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionary.java Thu Jan  5 14:58:50 2012
@@ -17,7 +17,6 @@ package org.apache.lucene.analysis.kurom
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 
 public class UnknownDictionary extends TokenInfoDictionary {
@@ -28,31 +27,6 @@ public class UnknownDictionary extends T
   
   private CharacterDefinition characterDefinition;
   
-  /**
-   * Constructor
-   */
-  public UnknownDictionary() {
-  }
-  
-  public UnknownDictionary(int size) {
-    super(size);
-    characterDefinition = new CharacterDefinition();    	
-  }
-  
-  @Override
-  public int put(String[] entry) {
-    // Get wordId of current entry
-    int wordId = buffer.position();
-    
-    // Put entry
-    int result = super.put(entry);
-    
-    // Put entry in targetMap
-    int characterId = CharacterDefinition.lookupCharacterClass(entry[0]);
-    addMapping(characterId, wordId);
-    return result;
-  }
-  
   public int lookup(String text) {
     if(!characterDefinition.isGroup(text.charAt(0))) {
       return 1;
@@ -72,37 +46,10 @@ public class UnknownDictionary extends T
     return length;
   }
   
-  /**
-   * Put mapping from unicode code point to character class.
-   * 
-   * @param codePoint code point
-   * @param characterClassName character class name
-   */
-  public void putCharacterCategory(int codePoint, String characterClassName) {
-    characterDefinition.putCharacterCategory(codePoint, characterClassName);
-  }
-  
-  public void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
-    characterDefinition.putInvokeDefinition(characterClassName, invoke, group, length);
-  }
-  
-  
   public CharacterDefinition getCharacterDefinition() {
     return characterDefinition;
   }
   
-  /**
-   * Write dictionary in file
-   * 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...].....
-   * @throws IOException
-   */
-  public void write(String directoryname) throws IOException {
-    writeDictionary(directoryname + File.separator + FILENAME);
-    writeTargetMap(directoryname + File.separator + TARGETMAP_FILENAME);
-    characterDefinition.write(directoryname);
-  }
-  
   public static UnknownDictionary getInstance() throws IOException, ClassNotFoundException {
     UnknownDictionary dictionary = new UnknownDictionary();
     dictionary.characterDefinition = CharacterDefinition.getInstance();

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java Thu Jan  5 14:58:50 2012
@@ -41,7 +41,7 @@ public class DictionaryBuilder {
       boolean normalizeEntry) throws IOException {
     System.out.println("building tokeninfo dict...");
     TokenInfoDictionaryBuilder tokenInfoBuilder = new TokenInfoDictionaryBuilder(format, encoding, normalizeEntry);
-    TokenInfoDictionary tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
+    TokenInfoDictionaryWriter tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
     
     System.out.print("  building double array trie...");
     DoubleArrayTrie trie = DoubleArrayTrieBuilder.build(tokenInfoBuilder.entrySet());
@@ -65,7 +65,7 @@ public class DictionaryBuilder {
     
     System.out.print("building unknown word dict...");
     UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
-    UnknownDictionary unkDictionary = unkBuilder.build(inputDirname);
+    UnknownDictionaryWriter unkDictionary = unkBuilder.build(inputDirname);
     unkDictionary.write(outputDirname+File.separatorChar+UnknownDictionary.class.getPackage().getName().replace('.',File.separatorChar));
     System.out.println("done");
     

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java Thu Jan  5 14:58:50 2012
@@ -33,7 +33,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
-import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
 import org.apache.lucene.analysis.kuromoji.util.DictionaryBuilder.DictionaryFormat;
 
 import com.ibm.icu.text.Normalizer2;
@@ -66,7 +65,7 @@ public class TokenInfoDictionaryBuilder 
     this.normalizer = normalizeEntries ? Normalizer2.getInstance(null, "nfkc", Normalizer2.Mode.COMPOSE) : null;
   }
   
-  public TokenInfoDictionary build(String dirname) throws IOException {
+  public TokenInfoDictionaryWriter build(String dirname) throws IOException {
     FilenameFilter filter = new FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
@@ -81,8 +80,8 @@ public class TokenInfoDictionaryBuilder 
     return buildDictionary(csvFiles);
   }
   
-  public TokenInfoDictionary buildDictionary(List<File> csvFiles) throws IOException {
-    TokenInfoDictionary dictionary = new TokenInfoDictionary(10 * 1024 * 1024);
+  public TokenInfoDictionaryWriter buildDictionary(List<File> csvFiles) throws IOException {
+    TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
     
     for (File file : csvFiles){
       FileInputStream inputStream = new FileInputStream(file);

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java?rev=1227618&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java Thu Jan  5 14:58:50 2012
@@ -0,0 +1,165 @@
+package org.apache.lucene.analysis.kuromoji.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.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.CodecUtil;
+
+import org.apache.lucene.analysis.kuromoji.dict.Dictionary;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+
+public class TokenInfoDictionaryWriter {
+  protected ByteBuffer buffer;
+  protected int[][] targetMap = new int[1][];
+  
+  public TokenInfoDictionaryWriter(int size) {
+    targetMap = new int[1][];
+    buffer = ByteBuffer.allocate(size);
+  }
+  
+  /**
+   * put the entry in map
+   * @return current position of buffer, which will be wordId of next entry
+   */
+  public int put(String[] entry) {
+    short leftId = Short.parseShort(entry[1]);
+    short rightId = Short.parseShort(entry[2]);
+    short wordCost = Short.parseShort(entry[3]);
+    
+    StringBuilder sb = new StringBuilder();
+    for (int i = 4; i < entry.length; i++){
+      sb.append(entry[i]).append(Dictionary.INTERNAL_SEPARATOR);
+    }
+    String features = sb.deleteCharAt(sb.length() - 1).toString();
+    int featuresSize = features.length()* 2;
+    
+    // extend buffer if necessary
+    int left = buffer.limit() - buffer.position();
+    if (8 + featuresSize > left) { // four short and features
+      ByteBuffer newBuffer = ByteBuffer.allocate(buffer.limit() * 2);
+      buffer.flip();
+      newBuffer.put(buffer);
+      buffer = newBuffer;
+    }
+    
+    buffer.putShort(leftId);
+    buffer.putShort(rightId);
+    buffer.putShort(wordCost);
+    buffer.putShort((short)featuresSize);
+    for (char c : features.toCharArray()){
+      buffer.putChar(c);
+    }
+    
+    return buffer.position();
+  }
+  
+  public void addMapping(int sourceId, int wordId) {
+    if(targetMap.length <= sourceId) {
+      int[][] newArray = new int[sourceId + 1][];
+      System.arraycopy(targetMap, 0, newArray, 0, targetMap.length);
+      targetMap = newArray;
+    }
+    
+    // Prepare array -- extend the length of array by one
+    int[] current = targetMap[sourceId];
+    if (current == null) {
+      current = new int[1];
+    } else {
+      int[] newArray = new int[current.length + 1];
+      System.arraycopy(current, 0, newArray, 0, current.length);
+      current = newArray;
+    }
+    targetMap[sourceId] = current;
+    
+    int[] targets = targetMap[sourceId];
+    targets[targets.length - 1] = wordId;
+  }
+
+  /**
+   * Write dictionary in file
+   * 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...].....
+   * @throws IOException
+   */
+  public void write(String directoryname) throws IOException {
+    writeDictionary(directoryname + File.separator + TokenInfoDictionary.FILENAME);
+    writeTargetMap(directoryname + File.separator + TokenInfoDictionary.TARGETMAP_FILENAME);
+  }
+  
+  protected void writeTargetMap(String filename) throws IOException {
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, TokenInfoDictionary.TARGETMAP_HEADER, TokenInfoDictionary.VERSION);
+      out.writeVInt(targetMap.length);
+      int nulls = 0;
+      for (int[] a : targetMap) {
+        if (a == null) {
+          // run-length encoding for all nulls:
+          if (nulls == 0) {
+            out.writeVInt(0);
+          }
+          nulls++;
+        } else {
+          if (nulls > 0) {
+            out.writeVInt(nulls);
+            nulls = 0;
+          }
+          assert a.length > 0;
+          out.writeVInt(a.length);
+          for (int i = 0; i < a.length; i++) {
+            out.writeVInt(a[i]);
+          }
+        }
+      }
+      // write the pending RLE count:
+      if (nulls > 0) {
+        out.writeVInt(nulls);
+      }
+    } finally {
+      os.close();
+    }
+  }
+  
+  protected void writeDictionary(String filename) throws IOException {
+    final FileOutputStream os = new FileOutputStream(filename);
+    try {
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, TokenInfoDictionary.DICT_HEADER, TokenInfoDictionary.VERSION);
+      out.writeVInt(buffer.position());
+      final WritableByteChannel channel = Channels.newChannel(os);
+      // Write Buffer
+      buffer.flip();  // set position to 0, set limit to current position
+      channel.write(buffer);
+      assert buffer.remaining() == 0L;
+    } finally {
+      os.close();
+    }
+  }
+}

Modified: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java?rev=1227618&r1=1227617&r2=1227618&view=diff
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java (original)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java Thu Jan  5 14:58:50 2012
@@ -26,8 +26,6 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 
-import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
-
 public class UnknownDictionaryBuilder {
   private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,5,5,-32768,-,*,*,*,*,*,*";
   
@@ -41,21 +39,20 @@ public class UnknownDictionaryBuilder {
     this.encoding = encoding;
   }
   
-  public UnknownDictionary build(String dirname) throws IOException {
-    UnknownDictionary unkDictionary = null;
-    unkDictionary = readDictionaryFile(dirname + File.separator + "unk.def");  //Should be only one file
+  public UnknownDictionaryWriter build(String dirname) throws IOException {
+    UnknownDictionaryWriter unkDictionary = readDictionaryFile(dirname + File.separator + "unk.def");  //Should be only one file
     readCharacterDefinition(dirname + File.separator + "char.def", unkDictionary);
     return unkDictionary;
   }
   
-  public UnknownDictionary readDictionaryFile(String filename)
+  public UnknownDictionaryWriter readDictionaryFile(String filename)
       throws IOException {
     return readDictionaryFile(filename, encoding);
   }
   
-  public UnknownDictionary readDictionaryFile(String filename, String encoding)
+  public UnknownDictionaryWriter readDictionaryFile(String filename, String encoding)
       throws IOException {
-    UnknownDictionary dictionary = new UnknownDictionary(5 * 1024 * 1024);
+    UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
     
     FileInputStream inputStream = new FileInputStream(filename);
     Charset cs = Charset.forName(encoding);
@@ -75,7 +72,7 @@ public class UnknownDictionaryBuilder {
     return dictionary;
   }
   
-  public void readCharacterDefinition(String filename, UnknownDictionary dictionary) throws IOException {
+  public void readCharacterDefinition(String filename, UnknownDictionaryWriter dictionary) throws IOException {
     FileInputStream inputStream = new FileInputStream(filename);
     InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
     LineNumberReader lineReader = new LineNumberReader(streamReader);

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java?rev=1227618&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java Thu Jan  5 14:58:50 2012
@@ -0,0 +1,55 @@
+package org.apache.lucene.analysis.kuromoji.util;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+
+public class UnknownDictionaryWriter extends TokenInfoDictionaryWriter {
+  private final CharacterDefinition characterDefinition = new CharacterDefinition();
+  
+  public UnknownDictionaryWriter(int size) {
+    super(size);
+  }
+  
+  @Override
+  public int put(String[] entry) {
+    // Get wordId of current entry
+    int wordId = buffer.position();
+    
+    // Put entry
+    int result = super.put(entry);
+    
+    // Put entry in targetMap
+    int characterId = CharacterDefinition.lookupCharacterClass(entry[0]);
+    addMapping(characterId, wordId);
+    return result;
+  }
+  
+  /**
+   * Put mapping from unicode code point to character class.
+   * 
+   * @param codePoint code point
+   * @param characterClassName character class name
+   */
+  public void putCharacterCategory(int codePoint, String characterClassName) {
+    characterDefinition.putCharacterCategory(codePoint, characterClassName);
+  }
+  
+  public void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
+    characterDefinition.putInvokeDefinition(characterClassName, invoke, group, length);
+  }
+  
+  /**
+   * Write dictionary in file
+   * 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...].....
+   * @throws IOException
+   */
+  public void write(String directoryname) throws IOException {
+    writeDictionary(directoryname + File.separator + UnknownDictionary.FILENAME);
+    writeTargetMap(directoryname + File.separator + UnknownDictionary.TARGETMAP_FILENAME);
+    characterDefinition.write(directoryname);
+  }
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java?rev=1227618&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java Thu Jan  5 14:58:50 2012
@@ -0,0 +1,75 @@
+package org.apache.lucene.analysis.kuromoji.dict;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.analysis.kuromoji.util.CSVUtil;
+import org.apache.lucene.analysis.kuromoji.util.UnknownDictionaryWriter;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+public class UnknownDictionaryTest extends LuceneTestCase {
+  public static final String FILENAME = "unk-tokeninfo-dict.obj";
+  
+  @Test
+  public void testPutCharacterCategory() {
+    UnknownDictionaryWriter unkDic = new UnknownDictionaryWriter(10 * 1024 * 1024);
+    
+    try{
+      unkDic.putCharacterCategory(0, "DUMMY_NAME");
+      fail();
+    } catch(Exception e) {
+      
+    }
+    
+    try{
+      unkDic.putCharacterCategory(-1, "KATAKANA");
+      fail();
+    } catch(Exception e) {
+      
+    }
+    
+    unkDic.putCharacterCategory(0, "DEFAULT");
+    unkDic.putCharacterCategory(1, "GREEK");
+    unkDic.putCharacterCategory(2, "HIRAGANA");
+    unkDic.putCharacterCategory(3, "KATAKANA");
+    unkDic.putCharacterCategory(4, "KANJI");
+  }
+  
+  @Test
+  public void testPut() {
+    UnknownDictionaryWriter unkDic = new UnknownDictionaryWriter(10 * 1024 * 1024);
+    try{
+      unkDic.put(CSVUtil.parse("KANJI,1285,11426,名詞,一般,*,*,*,*,*"));
+      fail();
+    } catch(Exception e){
+      
+    }
+    
+    String entry1 = "KANJI,1285,1285,11426,名詞,一般,*,*,*,*,*";
+    String entry2 = "ALPHA,1285,1285,13398,名詞,一般,*,*,*,*,*";
+    String entry3 = "HIRAGANA,1285,1285,13069,名詞,一般,*,*,*,*,*";
+    
+    unkDic.putCharacterCategory(0, "KANJI");
+    unkDic.putCharacterCategory(1, "ALPHA");
+    unkDic.putCharacterCategory(2, "HIRAGANA");
+    
+    unkDic.put(CSVUtil.parse(entry1));
+    unkDic.put(CSVUtil.parse(entry2));
+    unkDic.put(CSVUtil.parse(entry3));
+  }
+}