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/03 05:23:04 UTC

svn commit: r1226632 [2/2] - in /lucene/dev/branches/lucene3305: dev-tools/eclipse/ modules/analysis/ modules/analysis/kuromoji/ modules/analysis/kuromoji/src/ modules/analysis/kuromoji/src/java/ modules/analysis/kuromoji/src/java/org/ modules/analysis...

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsBuilder.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsBuilder.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsBuilder.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,63 @@
+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.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+
+public class ConnectionCostsBuilder {
+	
+	public ConnectionCostsBuilder() {
+		
+	}
+	
+	public static ConnectionCosts build(String filename) throws IOException {
+		FileInputStream inputStream = new FileInputStream(filename);
+		InputStreamReader streamReader = new InputStreamReader(inputStream);
+		LineNumberReader lineReader = new LineNumberReader(streamReader);
+
+		String line = lineReader.readLine();
+		String[] dimensions = line.split("\\s+");
+		
+		assert dimensions.length == 3;
+
+		int forwardSize = Integer.parseInt(dimensions[0]);
+		int backwardSize = Integer.parseInt(dimensions[1]);
+		
+		assert forwardSize > 0 && backwardSize > 0;
+		
+		ConnectionCosts costs = new ConnectionCosts(forwardSize, backwardSize);
+		
+		while ((line = lineReader.readLine()) != null) {
+			String[] fields = line.split("\\s+");
+
+			assert fields.length == 3;
+			
+			int forwardId = Integer.parseInt(fields[0]);
+			int backwardId = Integer.parseInt(fields[1]);
+			int cost = Integer.parseInt(fields[2]);
+
+			costs.add(forwardId, backwardId, cost);
+		}
+		return costs;
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,107 @@
+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.File;
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
+
+public class DictionaryBuilder {
+	
+	public enum DictionaryFormat { IPADIC, UNIDIC };
+	
+	public DictionaryBuilder() {
+		
+	}
+	
+	public void build(DictionaryFormat format,
+					  String inputDirname,
+					  String outputDirname,
+					  String encoding,
+					  boolean normalizeEntry) throws IOException {
+		System.out.println("building tokeninfo dict...");
+		TokenInfoDictionaryBuilder tokenInfoBuilder = new TokenInfoDictionaryBuilder(format, encoding, normalizeEntry);
+		TokenInfoDictionary tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
+
+		System.out.print("  building double array trie...");
+		DoubleArrayTrie trie = DoubleArrayTrieBuilder.build(tokenInfoBuilder.entrySet());
+		trie.write(outputDirname);
+		System.out.println("  done");
+
+		System.out.print("  processing target map...");
+		for (Entry<Integer, String> entry : tokenInfoBuilder.entrySet()) {
+			int tokenInfoId = entry.getKey();
+			String surfaceform = entry.getValue();
+			int doubleArrayId = trie.lookup(surfaceform);
+			assert doubleArrayId > 0;
+			tokenInfoDictionary.addMapping(doubleArrayId, tokenInfoId);
+		}		
+		tokenInfoDictionary.write(outputDirname);
+		trie = null;
+		tokenInfoBuilder = null;
+		
+		System.out.println("  done");
+		System.out.println("done");
+
+		System.out.print("building unknown word dict...");
+		UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
+		UnknownDictionary unkDictionary = unkBuilder.build(inputDirname);
+		unkDictionary.write(outputDirname);
+		System.out.println("done");
+
+		System.out.print("building connection costs...");
+		ConnectionCosts connectionCosts
+			= ConnectionCostsBuilder.build(inputDirname + File.separator + "matrix.def");
+		connectionCosts.write(outputDirname);
+		System.out.println("done");
+	}
+	
+	public static void main(String[] args) throws IOException, ClassNotFoundException {
+		DictionaryFormat format;
+		if (args[0].equalsIgnoreCase("ipadic")) {
+			format = DictionaryFormat.IPADIC;
+		} else if (args[0].equalsIgnoreCase("unidic")) {
+			format = DictionaryFormat.UNIDIC;
+		} else {
+			System.err.println("Illegal format " + args[0] + " using unidic instead");
+			format = DictionaryFormat.IPADIC;
+		}
+
+		String inputDirname = args[1];
+		String outputDirname = args[2];
+		String inputEncoding = args[3];
+		boolean normalizeEntries = Boolean.parseBoolean(args[4]);
+		
+		DictionaryBuilder builder = new DictionaryBuilder();
+		System.out.println("dictionary builder");
+		System.out.println("");
+		System.out.println("dictionary format: " + format);
+		System.out.println("input directory: " + inputDirname);
+		System.out.println("output directory: " + outputDirname);
+		System.out.println("input encoding: " + inputEncoding);
+		System.out.println("normalize entries: " + normalizeEntries);
+		System.out.println("");
+		builder.build(format, inputDirname, outputDirname, inputEncoding, normalizeEntries);
+	}
+	
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DoubleArrayTrieBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DoubleArrayTrieBuilder.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DoubleArrayTrieBuilder.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/DoubleArrayTrieBuilder.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,48 @@
+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.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
+import org.apache.lucene.analysis.kuromoji.trie.Trie;
+
+public class DoubleArrayTrieBuilder {
+
+	
+	public DoubleArrayTrieBuilder() {
+		
+	}
+
+	public static DoubleArrayTrie build(Set<Entry<Integer, String>> entries) {
+		Trie tempTrie = buildTrie(entries);
+		DoubleArrayTrie daTrie = new DoubleArrayTrie();
+		daTrie.build(tempTrie);
+		return daTrie;
+	}
+	
+	public static Trie buildTrie(Set<Entry<Integer, String>> entries) {
+		Trie trie = new Trie();
+		for (Entry<Integer, String> entry : entries) {
+			String surfaceForm = entry.getValue();
+			trie.add(surfaceForm);
+		}
+		return trie;
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,180 @@
+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.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.text.Normalizer;
+import java.util.ArrayList;
+import java.util.List;
+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;
+
+
+/**
+ * @author Masaru Hasegawa
+ * @author Christian Moen
+ */
+public class TokenInfoDictionaryBuilder {
+		
+	/** Internal word id - incrementally assigned as entries are read and added. This will be byte offset of dictionary file */
+	private int offset = 4; // Start from 4. First 4 bytes are used to store size of dictionary file.
+
+	private TreeMap<Integer, String> dictionaryEntries; // wordId, surface form
+
+	private String encoding = "euc-jp";
+	
+	private boolean normalizeEntries = false;
+	
+	private DictionaryFormat format = DictionaryFormat.IPADIC;
+	
+	public TokenInfoDictionaryBuilder() {
+	}
+	
+	public TokenInfoDictionaryBuilder(DictionaryFormat format, String encoding, boolean normalizeEntries) {
+		this.format = format;
+		this.encoding = encoding;
+		this.dictionaryEntries = new TreeMap<Integer, String>();		
+		this.normalizeEntries = normalizeEntries;
+	}
+
+	public TokenInfoDictionary build(String dirname) throws IOException {
+		FilenameFilter filter = new FilenameFilter() {
+			@Override
+			public boolean accept(File dir, String name) {
+				return name.endsWith(".csv");
+			}
+		};
+		ArrayList<File> csvFiles = new ArrayList<File>();
+		for (File file : new File(dirname).listFiles(filter)) {
+			csvFiles.add(file);
+		}
+		return buildDictionary(csvFiles);
+	}
+	
+	public TokenInfoDictionary buildDictionary(List<File> csvFiles) throws IOException {
+		TokenInfoDictionary dictionary = new TokenInfoDictionary(10 * 1024 * 1024);
+		
+		for (File file : csvFiles){
+			FileInputStream inputStream = new FileInputStream(file);
+			InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
+			BufferedReader reader = new BufferedReader(streamReader);
+
+			String line = null;
+			while ((line = reader.readLine()) != null) {
+				String[] entry = CSVUtil.parse(line);
+				if(entry.length < 13) {
+					System.out.println("Entry in CSV is not valid: " + line);
+					continue;
+				}
+				int next = dictionary.put(formatEntry(entry));
+
+				if(next == offset){
+					System.out.println("Failed to process line: " + line);
+					continue;
+				}
+				
+				dictionaryEntries.put(offset, entry[0]);
+				offset = next;
+				
+				// NFKC normalize dictionary entry
+				if (normalizeEntries) {
+					if (entry[0].equals(Normalizer.normalize(entry[0], Normalizer.Form.NFKC))){
+						continue;
+					}
+					String[] normalizedEntry = new String[entry.length];
+					for (int i = 0; i < entry.length; i++) {
+						normalizedEntry[i] = Normalizer.normalize(entry[i], Normalizer.Form.NFKC);
+					}
+					
+					next = dictionary.put(formatEntry(normalizedEntry));
+					dictionaryEntries.put(offset, normalizedEntry[0]);
+					offset = next;
+				}
+			}
+		}
+		
+		return dictionary;
+	}
+
+	/*
+	 * 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
+	 */
+	public String[] formatEntry(String[] features) {
+		if (this.format == 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;
+		}
+	}
+	
+	public Set<Entry<Integer, String>> entrySet() {
+		return dictionaryEntries.entrySet();
+	}	
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,113 @@
+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.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+
+public class UnknownDictionaryBuilder {
+	private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,5,5,-32768,-,*,*,*,*,*,*";
+	
+	private String encoding = "euc-jp";
+	
+	public UnknownDictionaryBuilder() {
+		
+	}
+
+	public UnknownDictionaryBuilder(String encoding) {
+		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
+		readCharacterDefinition(dirname + File.separator + "char.def", unkDictionary);
+		return unkDictionary;
+	}
+	
+	public UnknownDictionary readDictionaryFile(String filename)
+		throws IOException {
+		return readDictionaryFile(filename, encoding);
+	}
+
+	public UnknownDictionary readDictionaryFile(String filename, String encoding)
+		throws IOException {
+		UnknownDictionary dictionary = new UnknownDictionary(5 * 1024 * 1024);
+		
+		FileInputStream inputStream = new FileInputStream(filename);
+		InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
+		LineNumberReader lineReader = new LineNumberReader(streamReader);
+		
+		dictionary.put(CSVUtil.parse(NGRAM_DICTIONARY_ENTRY));
+
+		String line = null;
+		while ((line = lineReader.readLine()) != null) {
+			dictionary.put(CSVUtil.parse(line)); // Probably we don't need to validate entry
+		}
+
+		return dictionary;
+	}
+	
+	public void readCharacterDefinition(String filename, UnknownDictionary dictionary) throws IOException {
+		FileInputStream inputStream = new FileInputStream(filename);
+		InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
+		LineNumberReader lineReader = new LineNumberReader(streamReader);
+
+		String line = null;
+		
+		while ((line = lineReader.readLine()) != null) {
+			line = line.replaceAll("^\\s", "");
+			line = line.replaceAll("\\s*#.*", "");
+			line = line.replaceAll("\\s+", " ");
+			
+			// Skip empty line or comment line
+			if(line.length() == 0) {
+				continue;
+			}
+			
+			if(line.startsWith("0x")) {	// Category mapping
+				String[] values = line.split(" ", 2);	// Split only first space
+				
+				if(!values[0].contains("..")) {
+					int cp = Integer.decode(values[0]).intValue();
+					dictionary.putCharacterCategory(cp, values[1]);					
+				} else {
+					String[] codePoints = values[0].split("\\.\\.");
+					int cpFrom = Integer.decode(codePoints[0]).intValue();
+					int cpTo = Integer.decode(codePoints[1]).intValue();
+					
+					for(int i = cpFrom; i <= cpTo; i++){
+						dictionary.putCharacterCategory(i, values[1]);					
+					}
+				}
+			} else {	// Invoke definition
+				String[] values = line.split(" "); // Consecutive space is merged above
+				String characterClassName = values[0];
+				int invoke = Integer.parseInt(values[1]);
+				int group = Integer.parseInt(values[2]);
+				int length = Integer.parseInt(values[3]);
+				dictionary.putInvokeDefinition(characterClassName, invoke, group, length);
+			}
+		}
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/GraphvizFormatter.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,226 @@
+package org.apache.lucene.analysis.kuromoji.viterbi;
+
+/**
+ * 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
+
+public class GraphvizFormatter {
+
+	private final static String BOS_LABEL = "BOS";
+
+	private final static String EOS_LABEL = "EOS";
+	
+	private final static String FONT_NAME = "Helvetica";
+
+	private ConnectionCosts costs;
+	
+	private Map<String, ViterbiNode> nodeMap;
+
+	private Map<String, String> bestPathMap;
+
+	private boolean foundBOS;
+		
+	public GraphvizFormatter(ConnectionCosts costs) {
+		this.costs = costs;
+		this.nodeMap = new HashMap<String, ViterbiNode>();
+		this.bestPathMap = new HashMap<String, String>();
+	}
+	
+	public String format(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray) {
+		initBestPathMap(null);
+		
+		StringBuilder sb = new StringBuilder();
+		sb.append(formatHeader());
+		sb.append(formatNodes(startsArray, endsArray));
+		sb.append(formatTrailer());
+		return sb.toString();
+	}
+	
+	public String format(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray, List<ViterbiNode> bestPath) {
+
+//		List<ViterbiNode> bestPathWithBOSAndEOS = new ArrayList<ViterbiNode>(bastPath);
+		initBestPathMap(bestPath);
+
+		StringBuilder sb = new StringBuilder();
+		sb.append(formatHeader());
+		sb.append(formatNodes(startsArray, endsArray));
+		sb.append(formatTrailer());
+		return sb.toString();
+		
+	}
+	
+	private void initBestPathMap(List<ViterbiNode> bestPath) {
+		this.bestPathMap.clear();
+
+		if (bestPath == null){
+			return;
+		}
+		for (int i = 0; i < bestPath.size() - 1; i++) {
+			ViterbiNode from = bestPath.get(i);
+			ViterbiNode to = bestPath.get(i + 1);
+
+			String fromId = getNodeId(from);
+			String toId = getNodeId(to);
+			
+			assert this.bestPathMap.containsKey(fromId) == false;
+			assert this.bestPathMap.containsValue(toId) == false;
+			this.bestPathMap.put(fromId, toId);
+		}
+	}
+
+	private String formatNodes(ViterbiNode[][] startsArray, ViterbiNode[][] endsArray) {
+		this.nodeMap.clear();
+		this.foundBOS = false;
+		
+		StringBuilder sb = new StringBuilder();
+		for (int i = 1; i < endsArray.length; i++) {
+			if(endsArray[i] == null || startsArray[i] == null) {
+				continue;
+			}
+			for (int j = 0; j < endsArray[i].length; j++) {
+				ViterbiNode from = endsArray[i][j];
+				if(from == null){
+					continue;
+				}
+				sb.append(formatNodeIfNew(from));
+				for (int k = 0; k < startsArray[i].length; k++) {
+					ViterbiNode to = startsArray[i][k];
+					if(to == null){
+						break;
+					}
+					sb.append(formatNodeIfNew(to));
+					sb.append(formatEdge(from, to));
+				}
+			}
+		}
+		return sb.toString();
+	}
+	
+	private String formatNodeIfNew(ViterbiNode node) {
+		String nodeId = getNodeId(node);
+		if (! this.nodeMap.containsKey(nodeId)) {
+			this.nodeMap.put(nodeId, node);
+			return formatNode(node);
+		} else {
+			return "";
+		}
+	}	
+	
+	private String formatHeader() {
+		StringBuilder sb = new StringBuilder();
+		sb.append("digraph viterbi {\n");
+		sb.append("graph [ fontsize=30 labelloc=\"t\" label=\"\" splines=true overlap=false rankdir = \"LR\" ];\n");
+		sb.append("# A2 paper size\n");
+		sb.append("size = \"34.4,16.5\";\n");
+		sb.append("# try to fill paper\n");
+		sb.append("ratio = fill;\n");
+		sb.append("edge [ fontname=\"" + FONT_NAME + "\" fontcolor=\"red\" color=\"#606060\" ]\n");
+		sb.append("node [ style=\"filled\" fillcolor=\"#e8e8f0\" shape=\"Mrecord\" fontname=\"" + FONT_NAME + "\" ]\n");
+		
+		return sb.toString();
+	}
+
+	private String formatTrailer() {
+		return "}";
+	}
+	
+	
+	private String formatEdge(ViterbiNode from, ViterbiNode to) {
+		if (this.bestPathMap.containsKey(getNodeId(from)) &&
+			this.bestPathMap.get(getNodeId(from)).equals(getNodeId(to))) {
+			return formatEdge(from, to, "color=\"#40e050\" fontcolor=\"#40a050\" penwidth=3 fontsize=20 ");
+
+		} else {
+			return formatEdge(from, to, "");
+		}
+	}
+
+	
+	private String formatEdge(ViterbiNode from, ViterbiNode to, String attributes) {
+		StringBuilder sb = new StringBuilder();
+		sb.append(getNodeId(from));
+		sb.append(" -> ");
+		sb.append(getNodeId(to));
+		sb.append(" [ ");
+		sb.append("label=\"");
+		sb.append(getCost(from, to));
+		sb.append("\"");
+		sb.append(" ");
+		sb.append(attributes);
+		sb.append(" ");
+		sb.append(" ]");
+		sb.append("\n");
+		return sb.toString();
+	}
+	
+	private String formatNode(ViterbiNode node) {
+		StringBuilder sb = new StringBuilder();
+		sb.append("\"");
+		sb.append(getNodeId(node));
+		sb.append("\"");
+		sb.append(" [ ");
+		sb.append("label=");
+		sb.append(formatNodeLabel(node));
+		sb.append(" ]");
+		return sb.toString();
+	}
+	
+	private String formatNodeLabel(ViterbiNode node) {
+		StringBuilder sb = new StringBuilder();
+		sb.append("<<table border=\"0\" cellborder=\"0\">");
+		sb.append("<tr><td>");
+		sb.append(getNodeLabel(node));
+		sb.append("</td></tr>");
+		sb.append("<tr><td>");
+		sb.append("<font color=\"blue\">");
+		sb.append(node.getWordCost());
+		sb.append("</font>");
+		sb.append("</td></tr>");
+//		sb.append("<tr><td>");
+//		sb.append(this.dictionary.get(node.getWordId()).getPosInfo());
+//		sb.append("</td></tr>");
+		sb.append("</table>>");
+		return sb.toString();
+	}
+
+	private String getNodeId(ViterbiNode node) {
+		return String.valueOf(node.hashCode());
+	}
+	
+	private String getNodeLabel(ViterbiNode node) {
+		if (node.getType() == Type.KNOWN && node.getWordId() == 0) {
+			if (this.foundBOS) {
+				return EOS_LABEL;
+			} else {
+				this.foundBOS = true;
+				return BOS_LABEL;
+			}
+		} else {
+			return node.getSurfaceForm();
+		}
+	}
+	
+	private int getCost(ViterbiNode from, ViterbiNode to) {
+		return this.costs.get(from.getLeftId(), to.getRightId());
+	}
+}
\ No newline at end of file

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/Viterbi.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,353 @@
+package org.apache.lucene.analysis.kuromoji.viterbi;
+
+/**
+ * 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.util.LinkedList;
+import java.util.List;
+
+import org.apache.lucene.analysis.kuromoji.Tokenizer.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
+import org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition.CharacterClass;
+import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
+import org.apache.lucene.analysis.kuromoji.viterbi.ViterbiNode.Type;
+
+public class Viterbi {
+
+	private final DoubleArrayTrie trie;
+	
+	private final TokenInfoDictionary dictionary;
+	
+	private final UnknownDictionary unkDictionary;
+	
+	private final ConnectionCosts costs;
+	
+	private final UserDictionary userDictionary;
+	
+	private final CharacterDefinition characterDefinition;
+	
+	private final boolean useUserDictionary;
+
+	private final boolean searchMode;
+	
+	private final boolean extendedMode;
+	
+	private static final int DEFAULT_COST = 10000000;
+
+	private static final int SEARCH_MODE_LENGTH_KANJI = 3;
+
+	private static final int SEARCH_MODE_LENGTH = 7;
+
+	private static final int SEARCH_MODE_PENALTY = 10000;
+		
+	private static final String BOS = "BOS";
+	
+	private static final String EOS = "EOS";
+
+	/**
+	 * Constructor
+	 * @param trie
+	 * @param targetMap
+	 * @param dictionary
+	 * @param unkDictionary
+	 * @param costs
+	 * @param userDictionary
+	 */
+	public Viterbi(DoubleArrayTrie trie,
+				   TokenInfoDictionary dictionary,
+				   UnknownDictionary unkDictionary,
+				   ConnectionCosts costs,
+				   UserDictionary userDictionary,
+				   Mode mode) {
+		this.trie = trie;
+		this.dictionary = dictionary;
+		this.unkDictionary = unkDictionary;
+		this.costs = costs;
+		this.userDictionary = userDictionary;
+		if(userDictionary == null) {
+			this.useUserDictionary = false;
+		} else {
+			this.useUserDictionary = true;
+		}
+
+		switch(mode){
+		case SEARCH:
+			searchMode = true;
+			extendedMode = false;
+			break;
+		case EXTENDED:
+			searchMode = true;
+			extendedMode = true;
+			break;
+		default:
+			searchMode = false;
+			extendedMode = false;
+			break;
+		}
+
+		this.characterDefinition = unkDictionary.getCharacterDefinition();
+	}
+
+	/**
+	 * Find best path from input lattice.
+	 * @param lattice the result of build method
+	 * @return	List of ViterbiNode which consist best path 
+	 */
+	public List<ViterbiNode> search(ViterbiNode[][][] lattice) {
+		ViterbiNode[][] startIndexArr = lattice[0];
+		ViterbiNode[][] endIndexArr = lattice[1];
+		
+		for (int i = 1; i < startIndexArr.length; i++){
+
+			if (startIndexArr[i] == null || endIndexArr[i] == null){	// continue since no array which contains ViterbiNodes exists. Or no previous node exists.
+				continue;
+			}
+
+			for (ViterbiNode node : startIndexArr[i]) {
+				if (node == null){	// If array doesn't contain ViterbiNode any more, continue to next index
+					break;
+				}
+
+				int backwardConnectionId = node.getLeftId();
+				int wordCost = node.getWordCost();
+				int leastPathCost = DEFAULT_COST;
+				for (ViterbiNode leftNode : endIndexArr[i]) {
+					if (leftNode == null){ // If array doesn't contain ViterbiNode any more, continue to next index
+						break;
+					}
+					
+					int pathCost = leftNode.getPathCost() + costs.get(leftNode.getRightId(), backwardConnectionId) + wordCost;	// cost = [total cost from BOS to previous node] + [connection cost between previous node and current node] + [word cost]
+
+					// "Search mode". Add extra costs if it is long node.
+					if (searchMode) {
+//						System.out.print(""); // If this line exists, kuromoji runs faster for some reason when searchMode == false.
+						String surfaceForm = node.getSurfaceForm();
+						int length = surfaceForm.length();
+						if (length > SEARCH_MODE_LENGTH_KANJI) {
+							boolean allKanji = true;
+							// check if node consists of only kanji
+							for (int pos = 0; pos < length; pos++) {
+								if (!characterDefinition.isKanji(surfaceForm.charAt(pos))){
+									allKanji = false;
+									break;
+								}				
+							}
+							
+							if (allKanji) {	// Process only Kanji keywords
+								pathCost += (length - SEARCH_MODE_LENGTH_KANJI) * SEARCH_MODE_PENALTY;
+							} else if (length > SEARCH_MODE_LENGTH) {
+								pathCost += (length - SEARCH_MODE_LENGTH) * SEARCH_MODE_PENALTY;								
+							}
+						}
+					}
+					
+					if (pathCost < leastPathCost){	// If total cost is lower than before, set current previous node as best left node (previous means left).
+						leastPathCost = pathCost;
+						node.setPathCost(leastPathCost);
+						node.setLeftNode(leftNode);
+					}					
+				}
+			}
+		}
+
+		// track best path
+		ViterbiNode node = endIndexArr[0][0];	// EOS
+		LinkedList<ViterbiNode> result = new LinkedList<ViterbiNode>();
+		result.add(node);
+		while (true) {
+			ViterbiNode leftNode = node.getLeftNode();
+			if (leftNode == null) {
+				break;
+			}
+			
+			// EXTENDED mode convert unknown word into unigram node
+			if (extendedMode && leftNode.getType() == Type.UNKNOWN) {
+				int unigramWordId = CharacterClass.NGRAM.getId();
+				int unigramLeftId = unkDictionary.getLeftId(unigramWordId); // isn't required
+				int unigramRightId = unkDictionary.getLeftId(unigramWordId); // isn't required
+				int unigramWordCost = unkDictionary.getWordCost(unigramWordId); // isn't required
+				String surfaceForm = leftNode.getSurfaceForm();
+				for (int i = surfaceForm.length(); i > 0; i--) {
+					ViterbiNode uniGramNode = new ViterbiNode(unigramWordId, surfaceForm.substring(i - 1, i), unigramLeftId, unigramRightId, unigramWordCost, leftNode.getStartIndex() + i - 1, Type.UNKNOWN);
+					result.addFirst(uniGramNode);
+				}
+			} else {
+				result.addFirst(leftNode);		
+			}
+			node = leftNode;
+		}
+		
+		return result;
+	}
+	
+
+	/**
+	 * Build lattice from input text
+	 * @param text
+	 * @return
+	 */
+	public ViterbiNode[][][] build(String text) {
+		int textLength = text.length();
+		ViterbiNode[][] startIndexArr = new ViterbiNode[textLength + 2][];  // text length + BOS and EOS
+		ViterbiNode[][] endIndexArr = new ViterbiNode[textLength + 2][];  // text length + BOS and EOS
+		int[] startSizeArr = new int[textLength + 2]; // array to keep ViterbiNode count in startIndexArr
+		int[] endSizeArr = new int[textLength + 2];   // array to keep ViterbiNode count in endIndexArr
+		
+		ViterbiNode bosNode = new ViterbiNode(0, BOS, 0, 0, 0, -1, Type.KNOWN);
+		addToArrays(bosNode, 0, 1, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
+
+		// Process user dictionary;
+		if (useUserDictionary) {
+			processUserDictionary(text, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
+		}
+		
+		int unknownWordEndIndex = -1;	// index of the last character of unknown word
+
+		for (int startIndex = 0; startIndex < textLength; startIndex++) {
+			// If no token ends where current token starts, skip this index
+			if (endSizeArr[startIndex + 1] == 0) {
+				continue;
+			}
+			
+			String suffix = text.substring(startIndex);
+
+			boolean found = false;
+			for (int endIndex = 1; endIndex < suffix.length() + 1; endIndex++) {
+				String prefix = suffix.substring(0, endIndex);
+				
+				int result = trie.lookup(prefix);
+
+				if (result > 0) {	// Found match in double array trie
+					found = true;	// Don't produce unknown word starting from this index
+					for (int wordId : dictionary.lookupWordIds(result)) {
+						ViterbiNode node = new ViterbiNode(wordId, prefix, dictionary.getLeftId(wordId), dictionary.getRightId(wordId), dictionary.getWordCost(wordId), startIndex, Type.KNOWN);
+						addToArrays(node, startIndex + 1, startIndex + 1 + endIndex, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
+					}
+				} else if(result < 0) {	// If result is less than zero, continue to next position
+						break;						
+				}
+			}
+
+			// In the case of normal mode, it doesn't process unknown word greedily.
+			if(!searchMode && unknownWordEndIndex > startIndex){
+				continue;
+			}
+			
+			// Process Unknown Word
+			int unknownWordLength = 0;
+			char firstCharacter = suffix.charAt(0);
+			boolean isInvoke = characterDefinition.isInvoke(firstCharacter);
+			if (isInvoke){	// Process "invoke"
+				unknownWordLength = unkDictionary.lookup(suffix);
+			} else if (found == false){	// Process not "invoke"
+				unknownWordLength = unkDictionary.lookup(suffix);				
+			}
+			
+			if (unknownWordLength > 0) {      // found unknown word
+				String unkWord = suffix.substring(0, unknownWordLength);
+				int characterId = characterDefinition.lookup(firstCharacter);
+				int[] wordIds = unkDictionary.lookupWordIds(characterId); // characters in input text are supposed to be the same
+				
+				for (int wordId : wordIds) {
+					ViterbiNode node = new ViterbiNode(wordId, unkWord, unkDictionary.getLeftId(wordId), unkDictionary.getRightId(wordId), unkDictionary.getWordCost(wordId), startIndex, Type.UNKNOWN);
+					addToArrays(node, startIndex + 1, startIndex + 1 + unknownWordLength, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
+				}
+				unknownWordEndIndex = startIndex + unknownWordLength;
+			}
+		}
+		
+		ViterbiNode eosNode = new ViterbiNode(0, EOS, 0, 0, 0, textLength + 1, Type.KNOWN);
+		addToArrays(eosNode, textLength + 1, 0, startIndexArr, endIndexArr, startSizeArr, endSizeArr); //Add EOS node to endIndexArr at index 0
+		
+		ViterbiNode[][][] result = new ViterbiNode[][][]{startIndexArr, endIndexArr};
+		
+		return result;
+	}
+
+	/**
+	 * Find token(s) in input text and set found token(s) in arrays as normal tokens
+	 * @param text	
+	 * @param startIndexArr
+	 * @param endIndexArr
+	 * @param startSizeArr
+	 * @param endSizeArr
+	 */
+	private void processUserDictionary(String text, ViterbiNode[][] startIndexArr, ViterbiNode[][] endIndexArr, int[] startSizeArr, int[] endSizeArr) {
+		int[][] result = userDictionary.lookup(text);
+		for(int[] segmentation : result) {
+			int wordId = segmentation[0];
+			int index = segmentation[1];
+			int length = segmentation[2];
+			ViterbiNode node = new ViterbiNode(wordId, text.substring(index, index + length), userDictionary.getLeftId(wordId), userDictionary.getRightId(wordId), userDictionary.getWordCost(wordId), index, Type.USER);
+			addToArrays(node, index + 1, index + 1 + length, startIndexArr, endIndexArr, startSizeArr, endSizeArr);
+		}
+	}
+	
+	/**
+	 * Add node to arrays and increment count in size array
+	 * @param node
+	 * @param startIndex
+	 * @param endIndex
+	 * @param startIndexArr
+	 * @param endIndexArr
+	 * @param startSizeArr
+	 * @param endSizeArr
+	 */
+	private void addToArrays(ViterbiNode node, int startIndex, int endIndex, ViterbiNode[][] startIndexArr, ViterbiNode[][] endIndexArr, int[] startSizeArr, int[] endSizeArr ) {
+		int startNodesCount = startSizeArr[startIndex];
+		int endNodesCount = endSizeArr[endIndex];
+
+		if (startNodesCount == 0) {
+			startIndexArr[startIndex] = new ViterbiNode[10];
+		}
+
+		if (endNodesCount == 0) {
+			endIndexArr[endIndex] = new ViterbiNode[10];
+		}
+
+		if (startIndexArr[startIndex].length <= startNodesCount){
+			startIndexArr[startIndex] = extendArray(startIndexArr[startIndex]);
+		}
+		
+		if (endIndexArr[endIndex].length <= endNodesCount){
+			endIndexArr[endIndex] = extendArray(endIndexArr[endIndex]);
+		}
+				
+		startIndexArr[startIndex][startNodesCount] = node;
+		endIndexArr[endIndex][endNodesCount] = node;
+		
+		startSizeArr[startIndex] = startNodesCount + 1;
+		endSizeArr[endIndex] = endNodesCount + 1;
+	}
+	
+
+	/**
+	 * Return twice as big array which contains value of input array
+	 * @param array
+	 * @return
+	 */
+	private ViterbiNode[] extendArray(ViterbiNode[] array) {
+		//extend array
+		ViterbiNode[] newArray = new ViterbiNode[array.length * 2];
+		System.arraycopy(array, 0, newArray, 0, array.length);
+		return newArray;
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/java/org/apache/lucene/analysis/kuromoji/viterbi/ViterbiNode.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,122 @@
+package org.apache.lucene.analysis.kuromoji.viterbi;
+
+/**
+ * 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 ViterbiNode {
+	public enum Type {
+		KNOWN,
+		UNKNOWN,
+		USER
+	}
+
+	private final int wordId;
+	
+	private final String surfaceForm;
+	
+	private final int leftId;
+	
+	private final int rightId;
+	
+	/** word cost for this node */
+	private final int wordCost;
+	
+	/** minimum path cost found thus far */
+	private int pathCost;
+		
+	private ViterbiNode leftNode;
+
+	private final Type type;
+	
+	private final int startIndex;
+	
+	public ViterbiNode(int wordId, String surfaceForm, int leftId, int rightId, int wordCost, int startIndex, Type type) {
+		this.wordId = wordId;
+		this.surfaceForm = surfaceForm;
+		this.leftId = leftId;
+		this.rightId = rightId;
+		this.wordCost = wordCost;
+		this.startIndex = startIndex;
+		this.type = type;
+	}
+	
+
+	/**
+	 * @return the wordId
+	 */
+	public int getWordId() {
+		return wordId;
+	}
+
+	/**
+	 * @return the surfaceForm
+	 */
+	public String getSurfaceForm() {
+		return surfaceForm;
+	}
+
+	/**
+	 * @return the leftId
+	 */
+	public int getLeftId() {
+		return leftId;
+	}
+
+	/**
+	 * @return the rightId
+	 */
+	public int getRightId() {
+		return rightId;
+	}
+
+	/**
+	 * @return the cost
+	 */
+	public int getWordCost() {
+		return wordCost;
+	}
+
+	/**
+	 * @return the cost
+	 */
+	public int getPathCost() {
+		return pathCost;
+	}
+
+	/**
+	 * param cost minimum path cost found this far
+	 */
+	public void setPathCost(int pathCost) {
+		this.pathCost = pathCost;
+	}
+	
+	public void setLeftNode(ViterbiNode node) {
+		leftNode = node;
+	}
+
+	public ViterbiNode getLeftNode() {
+		return leftNode;
+	}
+
+	public int getStartIndex() {
+		return startIndex;
+	}
+
+	public Type getType() {
+		return type;
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/cc.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/cc.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/cd.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/cd.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/dat.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/dat.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/tid.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/tid.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/tid_map.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/tid_map.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/unk.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/unk.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/unk_map.dat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/resources/unk_map.dat?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TokenizerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TokenizerTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TokenizerTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/TokenizerTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,104 @@
+package org.apache.lucene.analysis.kuromoji;
+
+/**
+ * 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.IOException;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+import java.util.List;
+
+import org.apache.lucene.analysis.kuromoji.Token;
+import org.apache.lucene.analysis.kuromoji.Tokenizer;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TokenizerTest extends LuceneTestCase {
+
+	private static Tokenizer tokenizer;
+	
+	@BeforeClass
+	public static void setUpBeforeClass() throws Exception {
+		tokenizer = Tokenizer.builder().build();
+	}
+	
+	@AfterClass
+	public static void afterClass() throws Exception {
+	  tokenizer = null;
+	}
+
+	@Test
+	public void testSegmentation() {
+		// Skip tests for Michelle Kwan -- UniDic segments Kwan as ク ワン
+//		String input = "ミシェル・クワンが優勝しました。スペースステーションに行きます。うたがわしい。";
+//		String[] surfaceForms = {
+//				"ミシェル", "・", "クワン", "が", "優勝", "し", "まし", "た", "。",
+//				"スペース", "ステーション", "に", "行き", "ます", "。",
+//				"うたがわしい", "。"
+//		};
+		String input = "スペースステーションに行きます。うたがわしい。";
+		String[] surfaceForms = {
+				"スペース", "ステーション", "に", "行き", "ます", "。",
+				"うたがわしい", "。"
+		};
+		List<Token> tokens = tokenizer.tokenize(input);
+		assertTrue(tokens.size() == surfaceForms.length);
+		for (int i = 0; i < tokens.size(); i++) {
+			assertEquals(surfaceForms[i], tokens.get(i).getSurfaceForm());
+		}
+	}
+	
+	
+	@Test
+	public void testReadings() {
+		List<Token> tokens = tokenizer.tokenize("寿司が食べたいです。");
+		assertTrue(tokens.size() == 6);
+		assertEquals(tokens.get(0).getReading(), "スシ");
+		assertEquals(tokens.get(1).getReading(), "ガ");
+		assertEquals(tokens.get(2).getReading(), "タベ");
+		assertEquals(tokens.get(3).getReading(), "タイ");
+		assertEquals(tokens.get(4).getReading(), "デス");
+		assertEquals(tokens.get(5).getReading(), "。");
+	}
+	
+	@Test @Nightly
+	public void testBocchan() throws IOException, InterruptedException {
+		LineNumberReader reader = new LineNumberReader(new InputStreamReader(
+				this.getClass().getResourceAsStream("bocchan.utf-8")));
+		
+		String line = reader.readLine();
+		reader.close();
+
+		System.out.println("Test for Bocchan without pre-splitting sentences");
+		long totalStart = System.currentTimeMillis();
+		for (int i = 0; i < 100; i++){
+			tokenizer.tokenize(line);
+		}
+		System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+		System.out.println("Test for Bocchan with pre-splitting sentences");
+		String[] sentences = line.split("、|。");
+		totalStart = System.currentTimeMillis();
+		for (int i = 0; i < 100; i++) {
+			for (String sentence: sentences) {
+				tokenizer.tokenize(sentence);				
+			}
+		}
+		System.out.println("Total time : " + (System.currentTimeMillis() - totalStart));
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/bocchan.utf-8
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/bocchan.utf-8?rev=1226632&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/char.def.utf-8
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/char.def.utf-8?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/char.def.utf-8 (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/char.def.utf-8 Tue Jan  3 04:22:59 2012
@@ -0,0 +1,147 @@
+#
+#   Japanese charcter category map
+#
+#   $Id: char.def,v 1.4 2006/07/05 16:54:13 taku-ku Exp $;
+#
+
+###################################################################################
+# 
+#  CHARACTER CATEGORY DEFINITION
+#
+#  CATEGORY_NAME INVOKE GROUP LENGTH
+#
+#   - CATEGORY_NAME: Name of category. you have to define DEFAULT class.
+#   - INVOKE: 1/0:   always invoke unknown word processing, evan when the word can be found in the lexicon
+#   - GROUP:  1/0:   make a new word by grouping the same chracter category
+#   - LENGTH: n:     1 to n length new words are added
+#
+DEFAULT	       0 1 0  # DEFAULT is a mandatory category!
+SPACE	       0 1 0  
+KANJI	       0 0 2
+SYMBOL	       1 1 0
+NUMERIC	       1 1 0
+ALPHA	       1 1 0
+HIRAGANA       0 1 2 
+KATAKANA       1 1 2
+KANJINUMERIC   1 1 0
+GREEK	       1 1 0
+CYRILLIC       1 1 0
+
+###################################################################################
+#
+# CODE(UCS2) TO CATEGORY MAPPING
+#
+
+# SPACE
+0x0020 SPACE  # DO NOT REMOVE THIS LINE, 0x0020 is reserved for SPACE
+0x00D0 SPACE
+0x0009 SPACE
+0x000B SPACE
+0x000A SPACE
+
+# ASCII
+0x0021..0x002F SYMBOL
+0x0030..0x0039 NUMERIC
+0x003A..0x0040 SYMBOL
+0x0041..0x005A ALPHA
+0x005B..0x0060 SYMBOL
+0x0061..0x007A ALPHA
+0x007B..0x007E SYMBOL
+
+# Latin
+0x00A1..0x00BF SYMBOL # Latin 1
+0x00C0..0x00FF ALPHA  # Latin 1
+0x0100..0x017F ALPHA  # Latin Extended A
+0x0180..0x0236 ALPHA  # Latin Extended B
+0x1E00..0x1EF9 ALPHA  # Latin Extended Additional
+
+# CYRILLIC
+0x0400..0x04F9 CYRILLIC
+0x0500..0x050F CYRILLIC # Cyrillic supplementary
+
+# GREEK
+0x0374..0x03FB GREEK # Greek and Coptic
+
+# HIRAGANA
+0x3041..0x309F  HIRAGANA
+
+# KATAKANA
+0x30A1..0x30FF  KATAKANA
+0x31F0..0x31FF  KATAKANA  # Small KU .. Small RO
+# 0x30FC          KATAKANA HIRAGANA  # ー
+0x30FC          KATAKANA
+
+# Half KATAKANA
+0xFF66..0xFF9D  KATAKANA
+0xFF9E..0xFF9F  KATAKANA
+
+# KANJI
+0x2E80..0x2EF3  KANJI # CJK Raidcals Supplement
+0x2F00..0x2FD5  KANJI
+0x3005          KANJI
+0x3007          KANJI
+0x3400..0x4DB5  KANJI # CJK Unified Ideographs Extention
+0x4E00..0x9FA5  KANJI
+0xF900..0xFA2D  KANJI
+0xFA30..0xFA6A  KANJI
+
+# KANJI-NUMERIC (一 二 三 四 五 六 七 八 九 十 百 千 万 億 兆)
+0x4E00 KANJINUMERIC KANJI
+0x4E8C KANJINUMERIC KANJI
+0x4E09 KANJINUMERIC KANJI
+0x56DB KANJINUMERIC KANJI
+0x4E94 KANJINUMERIC KANJI
+0x516D KANJINUMERIC KANJI
+0x4E03 KANJINUMERIC KANJI
+0x516B KANJINUMERIC KANJI
+0x4E5D KANJINUMERIC KANJI
+0x5341 KANJINUMERIC KANJI
+0x767E KANJINUMERIC KANJI
+0x5343 KANJINUMERIC KANJI
+0x4E07 KANJINUMERIC KANJI
+0x5104 KANJINUMERIC KANJI
+0x5146 KANJINUMERIC KANJI
+
+# ZENKAKU 
+0xFF10..0xFF19 NUMERIC
+0xFF21..0xFF3A ALPHA
+0xFF41..0xFF5A ALPHA
+0xFF01..0xFF0F SYMBOL
+0xFF1A..0xFF1F SYMBOL
+0xFF3B..0xFF40 SYMBOL 
+0xFF5B..0xFF65 SYMBOL
+0xFFE0..0xFFEF SYMBOL # HalfWidth and Full width Form
+
+# OTHER SYMBOLS
+0x2000..0x206F  SYMBOL # General Punctuation
+0x2070..0x209F  NUMERIC # Superscripts and Subscripts
+0x20A0..0x20CF  SYMBOL # Currency Symbols
+0x20D0..0x20FF  SYMBOL # Combining Diaritical Marks for Symbols
+0x2100..0x214F  SYMBOL # Letterlike Symbols
+0x2150..0x218F  NUMERIC # Number forms
+0x2100..0x214B  SYMBOL # Letterlike Symbols
+0x2190..0x21FF  SYMBOL # Arrow
+0x2200..0x22FF  SYMBOL # Mathematical Operators
+0x2300..0x23FF  SYMBOL # Miscellaneuos Technical
+0x2460..0x24FF  SYMBOL # Enclosed NUMERICs
+0x2501..0x257F  SYMBOL # Box Drawing
+0x2580..0x259F  SYMBOL # Block Elements
+0x25A0..0x25FF  SYMBOL # Geometric Shapes
+0x2600..0x26FE  SYMBOL # Miscellaneous Symbols
+0x2700..0x27BF  SYMBOL # Dingbats
+0x27F0..0x27FF  SYMBOL # Supplemental Arrows A
+0x27C0..0x27EF  SYMBOL # Miscellaneous Mathematical Symbols-A
+0x2800..0x28FF  SYMBOL # Braille Patterns
+0x2900..0x297F  SYMBOL # Supplemental Arrows B
+0x2B00..0x2BFF  SYMBOL # Miscellaneous Symbols and Arrows
+0x2A00..0x2AFF  SYMBOL # Supplemental Mathematical Operators
+0x3300..0x33FF  SYMBOL
+0x3200..0x32FE  SYMBOL # ENclosed CJK Letters and Months
+0x3000..0x303F  SYMBOL # CJK Symbol and Punctuation
+0xFE30..0xFE4F  SYMBOL # CJK Compatibility Forms
+0xFE50..0xFE6B  SYMBOL # Small Form Variants
+
+# added 2006/3/13 
+0x3007 SYMBOL KANJINUMERIC
+
+# END OF TABLE

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,141 @@
+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 java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+import org.apache.lucene.analysis.kuromoji.util.CSVUtil;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class UnknownDictionaryTest extends LuceneTestCase {
+	public static final String FILENAME = "unk-tokeninfo-dict.obj";
+
+	@BeforeClass
+	public static void setUpBeforeClass() throws Exception {
+	}
+
+	@Test
+	public void testPutCharacterCategory() {
+		UnknownDictionary unkDic = new UnknownDictionary(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() {
+		UnknownDictionary unkDic = new UnknownDictionary(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));
+	}
+	
+	private UnknownDictionary createDictionary() throws IOException {
+		InputStream is = this.getClass().getClassLoader().getResourceAsStream("unk.def.utf-8");
+		UnknownDictionary dictionary = new UnknownDictionary();
+		BufferedReader reader = new BufferedReader(new InputStreamReader(is));
+		
+		String line = null;
+		while((line = reader.readLine()) != null) {
+			dictionary.put(CSVUtil.parse(line));
+		}
+		reader.close();
+
+		is = this.getClass().getClassLoader().getResourceAsStream("char.def.utf-8");
+		reader = new BufferedReader(new InputStreamReader(is));
+		
+		line = null;
+		while ((line = reader.readLine()) != null) {
+			line = line.replaceAll("^\\s", "");
+			line = line.replaceAll("\\s*#.*", "");
+			line = line.replaceAll("\\s+", " ");
+			
+			// Skip empty line or comment line
+			if(line.length() == 0) {
+				continue;
+			}
+			
+			if(line.startsWith("0x")) {	// Category mapping
+				String[] values = line.split(" ", 2);	// Split only first space
+				
+				if(!values[0].contains("..")) {
+					int cp = Integer.decode(values[0]).intValue();
+					dictionary.putCharacterCategory(cp, values[1]);					
+				} else {
+					String[] codePoints = values[0].split("\\.\\.");
+					int cpFrom = Integer.decode(codePoints[0]).intValue();
+					int cpTo = Integer.decode(codePoints[1]).intValue();
+					
+					for(int i = cpFrom; i <= cpTo; i++){
+						dictionary.putCharacterCategory(i, values[1]);					
+					}
+				}
+			} else {	// Invoke definition
+				String[] values = line.split(" "); // Consecutive space is merged above
+				String characterClassName = values[0];
+				int invoke = Integer.parseInt(values[1]);
+				int group = Integer.parseInt(values[2]);
+				int length = Integer.parseInt(values[3]);
+				dictionary.putInvokeDefinition(characterClassName, invoke, group, length);
+			}
+			
+		}
+		
+		reader.close();
+		
+		return dictionary;
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/dict/UserDictionaryTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,78 @@
+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 java.io.IOException;
+
+import org.apache.lucene.analysis.kuromoji.TokenizerTest;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class UserDictionaryTest {
+
+	@Test
+	public void testLookup() throws IOException {
+	 
+		UserDictionary dictionary = UserDictionary.read( TokenizerTest.class.getResourceAsStream("userdict.txt"));
+		int[][] dictionaryEntryResult = dictionary.lookup("関西国際空港に行った");
+		// Length should be three 関西, 国際, 空港
+		assertEquals(3, dictionaryEntryResult.length);
+
+		// Test positions
+		assertEquals(0, dictionaryEntryResult[0][1]); // index of 関西
+		assertEquals(2, dictionaryEntryResult[1][1]); // index of 国際
+		assertEquals(4, dictionaryEntryResult[2][1]); // index of 空港
+
+		// Test lengths
+		assertEquals(2, dictionaryEntryResult[0][2]); // length of 関西
+		assertEquals(2, dictionaryEntryResult[1][2]); // length of 国際
+		assertEquals(2, dictionaryEntryResult[2][2]); // length of 空港
+
+		int[][] dictionaryEntryResult2 = dictionary.lookup("関西国際空港と関西国際空港に行った");
+		// Length should be six 
+		assertEquals(6, dictionaryEntryResult2.length);
+	}
+
+	@Test
+	public void testReadings() throws IOException {
+    UserDictionary dictionary = UserDictionary.read( TokenizerTest.class.getResourceAsStream("userdict.txt"));
+		int wordIdNihon = 100000000; // wordId of 日本 in 日本経済新聞
+		assertEquals("ニホン", dictionary.getReading(wordIdNihon));
+
+		int wordIdAsashoryu = 100000006; // wordId for 朝青龍
+		assertEquals("アサショウリュウ", dictionary.getReading(wordIdAsashoryu));
+		
+		int wordIdNotExist = 1;
+		assertNull(dictionary.getReading(wordIdNotExist));
+	}
+	
+	@Test
+	public void testPartOfSpeech() throws IOException {
+    UserDictionary dictionary = UserDictionary.read( TokenizerTest.class.getResourceAsStream("userdict.txt"));
+		int wordIdKeizai = 100000001; // wordId of 経済 in 日本経済新聞
+		assertEquals("カスタム名詞", dictionary.getPartOfSpeech(wordIdKeizai));
+	}
+	
+	@Test
+	public void testRead() throws IOException {
+    UserDictionary dictionary = UserDictionary.read( TokenizerTest.class.getResourceAsStream("userdict.txt"));
+		assertNotNull(dictionary);		
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tokenizer.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tokenizer.properties?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tokenizer.properties (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/tokenizer.properties Tue Jan  3 04:22:59 2012
@@ -0,0 +1,10 @@
+##
+## tokenizer.properties
+##
+
+# Mecab IPADIC source dictionary
+mecab.ipadic.dir=/Users/cm/Projects/tmp5/mecab-ipadic-2.7.0-20070801
+
+# Tokenizer data dir
+tokenizer.data.dir=/Users/cm/Projects/tmp6
+

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrieTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrieTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrieTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/DoubleArrayTrieTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,105 @@
+package org.apache.lucene.analysis.kuromoji.trie;
+
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.kuromoji.trie.DoubleArrayTrie;
+import org.apache.lucene.analysis.kuromoji.trie.Trie;
+import org.junit.Test;
+
+public class DoubleArrayTrieTest {
+
+	@Test
+	public void buildTest() {		
+		Trie trie = getTrie();
+		DoubleArrayTrie doubleArrayTrie = new DoubleArrayTrie();
+		doubleArrayTrie.build(trie);
+	}
+
+	@Test
+	public void writeTest() throws IOException {
+		Trie trie = getTrie();
+		
+		DoubleArrayTrie doubleArrayTrie = new DoubleArrayTrie();
+		doubleArrayTrie.build(trie);
+		
+		try{
+			doubleArrayTrie.write("/some/path/which/is/not/exist");
+			fail();
+		}catch(IOException e){
+			
+		}
+		
+		String tmpDir = System.getProperty("java.io.tmpdir");
+		File dir = new File(tmpDir + File.separator + "datmp");
+		dir.mkdir();
+		doubleArrayTrie.write(dir.getCanonicalPath());
+		dir.deleteOnExit();
+		for(File file : dir.listFiles()) {
+			file.deleteOnExit();
+		}
+		
+		assertTrue(dir.length() > 0);
+		
+	}
+	
+	
+
+	@Test
+	public void lookupTest() throws IOException {
+		Trie trie = getTrie();
+		
+		DoubleArrayTrie doubleArrayTrie = new DoubleArrayTrie();
+		doubleArrayTrie.build(trie);
+
+		String tmpDir = System.getProperty("java.io.tmpdir");
+		File dir = new File(tmpDir + File.separator + "datmp");
+		dir.mkdir();
+		doubleArrayTrie.write(dir.getCanonicalPath());
+		dir.deleteOnExit();
+		for(File file : dir.listFiles()) {
+			file.deleteOnExit();
+		}
+
+		doubleArrayTrie = DoubleArrayTrie.read(new FileInputStream(dir.getCanonicalPath() + File.separator + DoubleArrayTrie.FILENAME));
+		
+		assertEquals(0, doubleArrayTrie.lookup("a"));
+		assertTrue(doubleArrayTrie.lookup("abc") > 0);
+		assertTrue(doubleArrayTrie.lookup("あいう") > 0);
+		assertTrue(doubleArrayTrie.lookup("xyz") < 0);
+
+	}
+	
+	private Trie getTrie() {
+		Trie trie = new Trie();
+		trie.add("abc");
+		trie.add("abd");
+		trie.add("あああ");
+		trie.add("あいう");
+		return trie;
+	}
+	
+
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/NodeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/NodeTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/NodeTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/NodeTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,165 @@
+package org.apache.lucene.analysis.kuromoji.trie;
+
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import org.apache.lucene.analysis.kuromoji.trie.Trie;
+import org.apache.lucene.analysis.kuromoji.trie.Trie.Node;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class NodeTest {
+
+	@BeforeClass
+	public static void setUpBeforeClass() throws Exception {
+	}
+
+	@Test
+	public void testNode() {
+		Trie trie = new Trie();
+		
+		Node node = trie.new Node('!');
+		assertEquals('!', node.getKey());
+
+		node = trie.new Node('1');
+		assertEquals('1', node.getKey());
+
+		node = trie.new Node('a');
+		assertEquals('a', node.getKey());
+		
+		node = trie.new Node('!');
+		assertEquals('!', node.getKey());
+
+		node = trie.new Node('1');
+		assertEquals('1', node.getKey());
+
+		node = trie.new Node('あ');
+		assertEquals('あ', node.getKey());
+
+		node = trie.new Node('æ¼¢');
+		assertEquals('æ¼¢', node.getKey());		
+		
+	}
+
+	@Test
+	public void testAddChild() {
+		Trie trie = new Trie();
+		Node node = trie.new Node('a');
+
+		Node returnedNode = node.addChild(trie.new Node('b'));
+		assertEquals('b', returnedNode.getKey());
+		assertEquals(1, node.getChildren().length);		
+		assertEquals('b', node.getChildren()[0].getKey());
+		
+		returnedNode = node.addChild(trie.new Node('c'));
+		assertEquals('c', returnedNode.getKey());		
+		assertEquals(2, node.getChildren().length);		
+		assertEquals('c', node.getChildren()[1].getKey());
+	}
+
+	@Test
+	public void testAdd() {
+		Trie trie = new Trie();
+
+		Node node = trie.new Node('a');
+		node.add("");
+		assertEquals(0, node.getChildren().length);
+		
+		node = trie.new Node('a');
+		node.add("b");
+		assertEquals(1, node.getChildren().length);
+		assertEquals('b', node.getChildren()[0].getKey());		
+
+		node = trie.new Node('a');
+		node.add("bc");
+		Node b = node.getChildren()[0];
+		assertEquals(1, node.getChildren().length);
+		assertEquals('b', b.getKey());		
+		assertEquals(1, b.getChildren().length);
+		Node c = b.getChildren()[0];
+		assertEquals('c', c.getKey());		
+		assertEquals(0, c.getChildren().length);
+
+		node.add("bd");
+		b = node.getChildren()[0];
+		assertEquals(1, node.getChildren().length);
+		assertEquals('b', b.getKey());
+		assertEquals(2, b.getChildren().length);
+		c = b.getChildren()[0];
+		assertEquals('c', c.getKey());		
+		assertEquals(0, c.getChildren().length);
+		Node d = b.getChildren()[1];
+		assertEquals('d', d.getKey());		
+		assertEquals(0, d.getChildren().length);
+	}
+	
+	
+	@Test
+	public void testGetkey() {
+		Trie trie = new Trie();
+
+		Node node = trie.new Node('!');
+		assertEquals('!', node.getKey());
+
+		node = trie.new Node('1');
+		assertEquals('1', node.getKey());
+
+		node = trie.new Node('a');
+		assertEquals('a', node.getKey());
+		
+		node = trie.new Node('!');
+		assertEquals('!', node.getKey());
+
+		node = trie.new Node('1');
+		assertEquals('1', node.getKey());
+
+		node = trie.new Node('あ');
+		assertEquals('あ', node.getKey());
+
+		node = trie.new Node('æ¼¢');
+		assertEquals('æ¼¢', node.getKey());		
+	}
+	
+	@Test
+	public void testHasSinglePath() {
+		Trie trie = new Trie();
+
+		Node node = trie.new Node('a');
+		node.add("bcd");
+		assertEquals(true, node.hasSinglePath());
+		
+		node.add("bce");
+		assertEquals(false, node.hasSinglePath());
+	}
+	
+	@Test
+	public void testGetChildren() {
+		Trie trie = new Trie();
+
+		Node node = trie.new Node('a');
+		node.add("bcd");
+		node.add("bde");
+		node.add("xyz");
+		
+		assertEquals(2, node.getChildren().length);
+		assertEquals('b', node.getChildren()[0].getKey());
+		assertEquals('x', node.getChildren()[1].getKey());
+		
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/TrieTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/TrieTest.java?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/TrieTest.java (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/trie/TrieTest.java Tue Jan  3 04:22:59 2012
@@ -0,0 +1,75 @@
+package org.apache.lucene.analysis.kuromoji.trie;
+
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.lucene.analysis.kuromoji.trie.Trie;
+import org.apache.lucene.analysis.kuromoji.trie.Trie.Node;
+import org.junit.Test;
+
+public class TrieTest {
+	
+	@Test
+	public void testGetRoot() {
+		Trie trie = new Trie();
+		Node rootNode = trie.getRoot();
+		assertNotNull(rootNode);
+	}
+	
+	@Test
+	public void testAdd() {
+		Trie trie = new Trie();
+		trie.add("aa");
+		trie.add("ab");
+		trie.add("bb");
+		
+		Node rootNode = trie.getRoot();
+		assertEquals(2, rootNode.getChildren().length);
+		assertEquals(2, rootNode.getChildren()[0].getChildren().length);
+		assertEquals(1, rootNode.getChildren()[1].getChildren().length);
+	}
+	
+	@Test
+	public void testGetChildren() {
+		Trie trie = new Trie();
+		trie.add("aa");
+		trie.add("ab");
+		trie.add("bb");
+		
+		Node rootNode = trie.getRoot();
+		assertEquals(2, rootNode.getChildren().length);
+		assertEquals(2, rootNode.getChildren()[0].getChildren().length);
+		assertEquals(1, rootNode.getChildren()[1].getChildren().length);
+	}
+	
+	@Test
+	public void testSinglePath() {
+		Trie trie = new Trie();
+		assertTrue(trie.getRoot().hasSinglePath());
+		trie.add("abcdef");
+		assertTrue(trie.getRoot().hasSinglePath());
+		trie.add("abdfg");
+		Node rootNode = trie.getRoot();
+		assertEquals(2, rootNode.getChildren()[0].getChildren()[0].getChildren().length);
+		assertTrue(rootNode.getChildren()[0].getChildren()[0].getChildren()[0].hasSinglePath());
+		assertTrue(rootNode.getChildren()[0].getChildren()[0].getChildren()[1].hasSinglePath());
+	}
+}

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/unk.def.utf-8
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/unk.def.utf-8?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/unk.def.utf-8 (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/unk.def.utf-8 Tue Jan  3 04:22:59 2012
@@ -0,0 +1,40 @@
+DEFAULT,5,5,4769,記号,一般,*,*,*,*,*
+SPACE,9,9,8903,記号,空白,*,*,*,*,*
+KANJI,1285,1285,11426,名詞,一般,*,*,*,*,*
+KANJI,1283,1283,17290,名詞,サ変接続,*,*,*,*,*
+KANJI,1293,1293,17611,名詞,固有名詞,地域,一般,*,*,*
+KANJI,1292,1292,12649,名詞,固有名詞,組織,*,*,*,*
+KANJI,1289,1289,17340,名詞,固有名詞,人名,一般,*,*,*
+KANJI,1288,1288,15295,名詞,固有名詞,一般,*,*,*,*
+SYMBOL,1283,1283,17585,名詞,サ変接続,*,*,*,*,*
+NUMERIC,1295,1295,27386,名詞,数,*,*,*,*,*
+ALPHA,1285,1285,13398,名詞,一般,*,*,*,*,*
+ALPHA,1293,1293,18706,名詞,固有名詞,地域,一般,*,*,*
+ALPHA,1292,1292,13835,名詞,固有名詞,組織,*,*,*,*
+ALPHA,1289,1289,18188,名詞,固有名詞,人名,一般,*,*,*
+ALPHA,1288,1288,15673,名詞,固有名詞,一般,*,*,*,*
+ALPHA,3,3,15235,感動詞,*,*,*,*,*,*
+HIRAGANA,1285,1285,13069,名詞,一般,*,*,*,*,*
+HIRAGANA,1283,1283,20223,名詞,サ変接続,*,*,*,*,*
+HIRAGANA,1293,1293,17882,名詞,固有名詞,地域,一般,*,*,*
+HIRAGANA,1292,1292,14761,名詞,固有名詞,組織,*,*,*,*
+HIRAGANA,1289,1289,18060,名詞,固有名詞,人名,一般,*,*,*
+HIRAGANA,1288,1288,14787,名詞,固有名詞,一般,*,*,*,*
+HIRAGANA,3,3,16989,感動詞,*,*,*,*,*,*
+KATAKANA,1285,1285,9461,名詞,一般,*,*,*,*,*
+KATAKANA,1293,1293,13661,名詞,固有名詞,地域,一般,*,*,*
+KATAKANA,1292,1292,10922,名詞,固有名詞,組織,*,*,*,*
+KATAKANA,1289,1289,13581,名詞,固有名詞,人名,一般,*,*,*
+KATAKANA,1288,1288,10521,名詞,固有名詞,一般,*,*,*,*
+KATAKANA,3,3,14138,感動詞,*,*,*,*,*,*
+KANJINUMERIC,1295,1295,27473,名詞,数,*,*,*,*,*
+GREEK,1285,1285,7884,名詞,一般,*,*,*,*,*
+GREEK,1293,1293,12681,名詞,固有名詞,地域,一般,*,*,*
+GREEK,1292,1292,8573,名詞,固有名詞,組織,*,*,*,*
+GREEK,1289,1289,12697,名詞,固有名詞,人名,一般,*,*,*
+GREEK,1288,1288,10029,名詞,固有名詞,一般,*,*,*,*
+CYRILLIC,1285,1285,7966,名詞,一般,*,*,*,*,*
+CYRILLIC,1293,1293,12600,名詞,固有名詞,地域,一般,*,*,*
+CYRILLIC,1292,1292,8492,名詞,固有名詞,組織,*,*,*,*
+CYRILLIC,1289,1289,12615,名詞,固有名詞,人名,一般,*,*,*
+CYRILLIC,1288,1288,9866,名詞,固有名詞,一般,*,*,*,*

Added: lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt?rev=1226632&view=auto
==============================================================================
--- lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt (added)
+++ lucene/dev/branches/lucene3305/modules/analysis/kuromoji/src/test/org/apache/lucene/analysis/kuromoji/userdict.txt Tue Jan  3 04:22:59 2012
@@ -0,0 +1,6 @@
+# Custom segmentation for long entries
+日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞
+関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,テスト名詞
+
+# Custom reading for sumo wrestler
+朝青龍,朝青龍,アサショウリュウ,カスタム人名