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/12 21:10:52 UTC

svn commit: r1230748 [5/5] - in /lucene/dev/trunk: dev-tools/eclipse/ lucene/contrib/ modules/analysis/ modules/analysis/common/src/java/org/apache/lucene/analysis/util/ modules/analysis/common/src/test/org/apache/lucene/analysis/util/ modules/analysis...

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsWriter.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsWriter.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/ConnectionCostsWriter.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,76 @@
+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 org.apache.lucene.analysis.kuromoji.dict.ConnectionCosts;
+
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.CodecUtil;
+
+public final class ConnectionCostsWriter {
+  
+  private final short[][] costs; // array is backward IDs first since get is called using the same backward ID consecutively. maybe doesn't matter.
+  private final int forwardSize;
+  private final int backwardSize;
+  /**
+   * Constructor for building. TODO: remove write access
+   */
+  public ConnectionCostsWriter(int forwardSize, int backwardSize) {
+    this.forwardSize = forwardSize;
+    this.backwardSize = backwardSize;
+    this.costs = new short[backwardSize][forwardSize];
+  }
+  
+  public void add(int forwardId, int backwardId, int cost) {
+    this.costs[backwardId][forwardId] = (short)cost;
+  }
+  
+  public void write(String baseDir) throws IOException {
+    String filename = baseDir + File.separator +
+      ConnectionCosts.class.getName().replace('.', File.separatorChar) + ConnectionCosts.FILENAME_SUFFIX;
+    new File(filename).getParentFile().mkdirs();
+    OutputStream os = new FileOutputStream(filename);
+    try {
+      os = new BufferedOutputStream(os);
+      final DataOutput out = new OutputStreamDataOutput(os);
+      CodecUtil.writeHeader(out, ConnectionCosts.HEADER, ConnectionCosts.VERSION);
+      out.writeVInt(forwardSize);
+      out.writeVInt(backwardSize);
+      int last = 0;
+      assert costs.length == backwardSize;
+      for (short[] a : costs) {
+        assert a.length == forwardSize;
+        for (int i = 0; i < a.length; i++) {
+          int delta = (int)a[i] - last;
+          out.writeVInt((delta >> 31) ^ (delta << 1));
+          last = a[i];
+        }
+      }
+    } finally {
+      os.close();
+    }
+  }
+  
+}

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/DictionaryBuilder.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,85 @@
+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;
+
+public class DictionaryBuilder {
+  
+  public enum DictionaryFormat { IPADIC, UNIDIC };
+  
+  private DictionaryBuilder() {
+  }
+  
+  public static 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);    
+    TokenInfoDictionaryWriter tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
+    tokenInfoDictionary.write(outputDirname);
+    tokenInfoDictionary = null;
+    tokenInfoBuilder = null;
+    System.out.println("done");
+    
+    System.out.print("building unknown word dict...");
+    UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
+    UnknownDictionaryWriter unkDictionary = unkBuilder.build(inputDirname);
+    unkDictionary.write(outputDirname);
+    unkDictionary = null;
+    unkBuilder = null;
+    System.out.println("done");
+    
+    System.out.print("building connection costs...");
+    ConnectionCostsWriter 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]);
+    
+    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("");
+    DictionaryBuilder.build(format, inputDirname, outputDirname, inputEncoding, normalizeEntries);
+  }
+  
+}

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryBuilder.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,230 @@
+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.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.analysis.kuromoji.util.DictionaryBuilder.DictionaryFormat;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+
+import com.ibm.icu.text.Normalizer2;
+
+/**
+ */
+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 = 0;
+  
+  private String encoding = "euc-jp";
+  
+  private boolean normalizeEntries = false;
+  private Normalizer2 normalizer;
+  
+  private DictionaryFormat format = DictionaryFormat.IPADIC;
+  
+  public TokenInfoDictionaryBuilder(DictionaryFormat format, String encoding, boolean normalizeEntries) {
+    this.format = format;
+    this.encoding = encoding;
+    this.normalizeEntries = normalizeEntries;
+    this.normalizer = normalizeEntries ? Normalizer2.getInstance(null, "nfkc", Normalizer2.Mode.COMPOSE) : null;
+  }
+  
+  public TokenInfoDictionaryWriter 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);
+    }
+    Collections.sort(csvFiles);
+    return buildDictionary(csvFiles);
+  }
+
+  public TokenInfoDictionaryWriter buildDictionary(List<File> csvFiles) throws IOException {
+    TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
+    
+    // all lines in the file
+    System.out.println("  parse...");
+    List<String[]> lines = new ArrayList<String[]>(400000);
+    for (File file : csvFiles){
+      FileInputStream inputStream = new FileInputStream(file);
+      Charset cs = Charset.forName(encoding);
+      CharsetDecoder decoder = cs.newDecoder()
+          .onMalformedInput(CodingErrorAction.REPORT)
+          .onUnmappableCharacter(CodingErrorAction.REPORT);
+      InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
+      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;
+        }
+        
+        String[] formatted = formatEntry(entry);
+        dictionary.noteInflection(formatted);
+        lines.add(formatted);
+        
+        // NFKC normalize dictionary entry
+        if (normalizeEntries) {
+          if (normalizer.isNormalized(entry[0])){
+            continue;
+          }
+          String[] normalizedEntry = new String[entry.length];
+          for (int i = 0; i < entry.length; i++) {
+            normalizedEntry[i] = normalizer.normalize(entry[i]);
+          }
+          
+          formatted = formatEntry(normalizedEntry);
+          dictionary.noteInflection(formatted);
+          lines.add(formatted);
+        }
+      }
+    }
+    
+    dictionary.finalizeInflections();
+    
+    System.out.println("  sort...");
+
+    // sort by term: we sorted the files already and use a stable sort.
+    Collections.sort(lines, new Comparator<String[]>() {
+      public int compare(String[] left, String[] right) {
+        return left[0].compareTo(right[0]);
+      }
+    });
+    
+    System.out.println("  encode...");
+
+    PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton(true);
+    Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, fstOutput);
+    IntsRef scratch = new IntsRef();
+    long ord = -1; // first ord will be 0
+    String lastValue = null;
+
+    // build tokeninfo dictionary
+    for (String[] entry : lines) {
+      int next = dictionary.put(entry);
+        
+      if(next == offset){
+        System.out.println("Failed to process line: " + Arrays.toString(entry));
+        continue;
+      }
+      
+      String token = entry[0];
+      if (!token.equals(lastValue)) {
+        // new word to add to fst
+        ord++;
+        lastValue = token;
+        scratch.grow(token.length());
+        scratch.length = token.length();
+        for (int i = 0; i < token.length(); i++) {
+          scratch.ints[i] = (int) token.charAt(i);
+        }
+        fstBuilder.add(scratch, fstOutput.get(ord));
+      }
+      dictionary.addMapping((int)ord, offset);
+      offset = next;
+    }
+    
+    FST<Long> fst = fstBuilder.finish();
+    System.out.print("  " + fst.getNodeCount() + " nodes, " + fst.getArcCount() + " arcs, " + fst.sizeInBytes() + " bytes...  ");
+    dictionary.setFST(fst);
+    System.out.println(" done");
+    
+    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;
+    }
+  }
+}

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/TokenInfoDictionaryWriter.java Thu Jan 12 20:10:48 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.io.File;
+import java.io.IOException;
+
+import org.apache.lucene.analysis.kuromoji.dict.TokenInfoDictionary;
+import org.apache.lucene.util.fst.FST;
+
+public class TokenInfoDictionaryWriter extends BinaryDictionaryWriter {
+  private FST<Long> fst;
+
+  public TokenInfoDictionaryWriter(int size) {
+    super(TokenInfoDictionary.class, size);
+  }
+  
+  public void setFST(FST<Long> fst) {
+    this.fst = fst;
+  }
+  
+  @Override
+  public void write(String baseDir) throws IOException {
+    super.write(baseDir);
+    writeFST(getBaseFileName(baseDir) + TokenInfoDictionary.FST_FILENAME_SUFFIX);
+  }
+  
+  protected void writeFST(String filename) throws IOException {
+    File f = new File(filename);
+    f.getParentFile().mkdirs();
+    fst.save(f);
+  }  
+}

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryBuilder.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,137 @@
+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 java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.analysis.kuromoji.dict.CharacterDefinition;
+
+public class UnknownDictionaryBuilder {
+  private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,5,5,-32768,-,*,*,*,*,*,*,*,*";
+  
+  private String encoding = "euc-jp";
+  
+  public UnknownDictionaryBuilder(String encoding) {
+    this.encoding = encoding;
+  }
+  
+  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 UnknownDictionaryWriter readDictionaryFile(String filename)
+      throws IOException {
+    return readDictionaryFile(filename, encoding);
+  }
+  
+  public UnknownDictionaryWriter readDictionaryFile(String filename, String encoding)
+      throws IOException {
+    UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
+    
+    FileInputStream inputStream = new FileInputStream(filename);
+    Charset cs = Charset.forName(encoding);
+    CharsetDecoder decoder = cs.newDecoder()
+        .onMalformedInput(CodingErrorAction.REPORT)
+        .onUnmappableCharacter(CodingErrorAction.REPORT);
+    InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
+    LineNumberReader lineReader = new LineNumberReader(streamReader);
+    
+    dictionary.put(CSVUtil.parse(NGRAM_DICTIONARY_ENTRY));
+    
+    List<String[]> lines = new ArrayList<String[]>();
+    String line = null;
+    while ((line = lineReader.readLine()) != null) {
+      // note: unk.def only has 10 fields, it simplifies the writer to just append empty reading and pronunciation,
+      // even though the unknown dictionary returns hardcoded null here.
+      final String[] parsed = CSVUtil.parse(line + ",*,*"); // Probably we don't need to validate entry
+      lines.add(parsed);
+      dictionary.noteInflection(parsed); // for completeness; I think unk.def has no inflections...
+    }
+    dictionary.finalizeInflections(); // should also be no-op
+    
+    Collections.sort(lines, new Comparator<String[]>() {
+      public int compare(String[] left, String[] right) {
+        int leftId = CharacterDefinition.lookupCharacterClass(left[0]);
+        int rightId = CharacterDefinition.lookupCharacterClass(right[0]);
+        return leftId - rightId;
+      }
+    });
+    
+    for (String[] entry : lines) {
+      dictionary.put(entry);
+    }
+    
+    return dictionary;
+  }
+  
+  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);
+    
+    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/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/kuromoji/util/UnknownDictionaryWriter.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,50 @@
+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.BinaryDictionary;
+import org.apache.lucene.analysis.kuromoji.dict.UnknownDictionary;
+
+public class UnknownDictionaryWriter extends BinaryDictionaryWriter {
+  private final CharacterDefinitionWriter characterDefinition = new CharacterDefinitionWriter();
+  
+  public UnknownDictionaryWriter(int size) {
+    super(UnknownDictionary.class, 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);
+  }
+  
+  @Override
+  public void write(String baseDir) throws IOException {
+    super.write(baseDir);
+    characterDefinition.write(baseDir);
+  }
+}

Added: lucene/dev/trunk/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java (added)
+++ lucene/dev/trunk/modules/analysis/kuromoji/src/tools/test/org/apache/lucene/analysis/kuromoji/dict/UnknownDictionaryTest.java Thu Jan 12 20:10:48 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 = "ALPHA,1285,1285,13398,名詞,一般,*,*,*,*,*,*,*";
+    String entry2 = "HIRAGANA,1285,1285,13069,名詞,一般,*,*,*,*,*,*,*";
+    String entry3 = "KANJI,1285,1285,11426,名詞,一般,*,*,*,*,*,*,*";
+
+    unkDic.putCharacterCategory(0, "ALPHA");
+    unkDic.putCharacterCategory(1, "HIRAGANA");
+    unkDic.putCharacterCategory(2, "KANJI");
+    
+    unkDic.put(CSVUtil.parse(entry1));
+    unkDic.put(CSVUtil.parse(entry2));
+    unkDic.put(CSVUtil.parse(entry3));
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/CHANGES.txt?rev=1230748&r1=1230747&r2=1230748&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/CHANGES.txt Thu Jan 12 20:10:48 2012
@@ -17,6 +17,9 @@ $Id$
   the Solr 3.x ICUCollationKeyFilterFactory, and also supports
   Locale-sensitive range queries.  (rmuir)
 
+* LUCENE-3305: Added Kuromoji morphological analyzer for Japanese.
+  (Christian Moen, Masaru Hasegawa via Robert Muir)
+
 ==================  3.6.0 ==================
 
 * SOLR-2919: Added parametric tailoring options to ICUCollationKeyFilterFactory.

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/README.txt?rev=1230748&r1=1230747&r2=1230748&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/README.txt (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/README.txt Thu Jan 12 20:10:48 2012
@@ -2,15 +2,16 @@ The analysis-extras plugin provides addi
 upon large dependencies/dictionaries.
 
 It includes integration with ICU for multilingual support, and 
-analyzers for Chinese and Polish.
+analyzers for Chinese, Japanese, and Polish.
 
 Relies upon the following lucene components (in lucene-libs/):
 
  * lucene-analyzers-icu-X.Y.jar
+ * lucene-analyzers-kuromoji-X.Y.jar
  * lucene-analyzers-smartcn-X.Y.jar
  * lucene-analyzers-stempel-X.Y.jar
  
 And the ICU library (in lib/):
 
  * icu4j-X.Y.jar
- 
\ No newline at end of file
+ 

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/build.xml?rev=1230748&r1=1230747&r2=1230748&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/build.xml (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/build.xml Thu Jan 12 20:10:48 2012
@@ -27,21 +27,23 @@
 
   <path id="classpath">
   	<pathelement path="${analyzers-icu.jar}"/>
+        <pathelement path="${analyzers-kuromoji.jar}"/>
   	<pathelement path="${analyzers-smartcn.jar}"/>
   	<pathelement path="${analyzers-stempel.jar}"/>
     <path refid="solr.base.classpath"/>
   </path>
 
   <target name="module-jars-to-solr"
-          depends="jar-analyzers-icu, jar-analyzers-smartcn, jar-analyzers-stempel">
+          depends="jar-analyzers-icu, jar-analyzers-kuromoji, jar-analyzers-smartcn, jar-analyzers-stempel">
     <mkdir dir="${build.dir}/lucene-libs"/>
     <copy todir="${build.dir}/lucene-libs" preservelastmodified="true" flatten="true" failonerror="true" overwrite="true">
       <fileset file="${analyzers-icu.jar}"/>
+      <fileset file="${analyzers-kuromoji.jar}"/>
       <fileset file="${analyzers-smartcn.jar}"/>
       <fileset file="${analyzers-stempel.jar}"/>
     </copy>
   </target>
 
-  <target name="compile-core" depends="jar-analyzers-icu, jar-analyzers-smartcn, jar-analyzers-stempel, solr-contrib-build.compile-core"/>
+  <target name="compile-core" depends="jar-analyzers-icu, jar-analyzers-kuromoji, jar-analyzers-smartcn, jar-analyzers-stempel, solr-contrib-build.compile-core"/>
   <target name="dist" depends="module-jars-to-solr, common-solr.dist"/>
 </project>

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiBaseFormFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiBaseFormFilterFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiBaseFormFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiBaseFormFilterFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,40 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.TokenStream;
+import org.apache.lucene.analysis.kuromoji.KuromojiBaseFormFilter;
+
+/**
+ * Factory for {@link KuromojiBaseFormFilter}.  
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_ja" class="solr.TextField"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.KuromojiTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.KuromojiBaseFormFilterFactory"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;
+ * </pre>
+ */
+public class KuromojiBaseFormFilterFactory extends BaseTokenFilterFactory {
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new KuromojiBaseFormFilter(input);
+  }
+}

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiPartOfSpeechStopFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiPartOfSpeechStopFilterFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiPartOfSpeechStopFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiPartOfSpeechStopFilterFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,65 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.kuromoji.KuromojiPartOfSpeechStopFilter;
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.util.plugin.ResourceLoaderAware;
+
+/**
+ * Factory for {@link KuromojiPartOfSpeechStopFilter}.  
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_ja" class="solr.TextField"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.KuromojiTokenizerFactory"/&gt;
+ *     &lt;filter class="solr.KuromojiPartOfSpeechStopFilterFactory" 
+ *             tags="stopTags.txt" 
+ *             enablePositionIncrements="true"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;
+ * </pre>
+ */
+public class KuromojiPartOfSpeechStopFilterFactory extends BaseTokenFilterFactory implements ResourceLoaderAware  {
+  private boolean enablePositionIncrements;
+  private Set<String> stopTags;
+
+  public void inform(ResourceLoader loader) {
+    String stopTagFiles = args.get("tags");
+    enablePositionIncrements = getBoolean("enablePositionIncrements", false);
+    try {
+      CharArraySet cas = getWordSet(loader, stopTagFiles, false);
+      stopTags = new HashSet<String>();
+      for (Object element : cas) {
+        char chars[] = (char[]) element;
+        stopTags.add(new String(chars));
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public TokenStream create(TokenStream stream) {
+    return new KuromojiPartOfSpeechStopFilter(enablePositionIncrements, stream, stopTags);
+  }
+}

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/KuromojiTokenizerFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,92 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.util.Locale;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.kuromoji.KuromojiTokenizer;
+import org.apache.lucene.analysis.kuromoji.Segmenter;
+import org.apache.lucene.analysis.kuromoji.Segmenter.Mode;
+import org.apache.lucene.analysis.kuromoji.dict.UserDictionary;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.analysis.BaseTokenizerFactory;
+import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.plugin.ResourceLoaderAware;
+
+/**
+ * Factory for {@link KuromojiTokenizer}.  
+ * <pre class="prettyprint">
+ * &lt;fieldType name="text_ja" class="solr.TextField"&gt;
+ *   &lt;analyzer&gt;
+ *     &lt;tokenizer class="solr.KuromojiTokenizerFactory"
+ *       mode=NORMAL
+ *       user-dictionary=user.txt
+ *       user-dictionary-encoding=UTF-8
+ *     /&gt;
+ *     &lt;filter class="solr.KuromojiBaseFormFilterFactory"/&gt;
+ *   &lt;/analyzer&gt;
+ * &lt;/fieldType&gt;
+ * </pre>
+ */
+public class KuromojiTokenizerFactory extends BaseTokenizerFactory implements ResourceLoaderAware {
+  private static final String MODE = "mode";
+  
+  private static final String USER_DICT_PATH = "user-dictionary";
+  
+  private static final String USER_DICT_ENCODING = "user-dictionary-encoding";
+
+  private Segmenter segmenter;
+  
+  @Override
+  public void inform(ResourceLoader loader) {
+    Mode mode = args.get(MODE) != null ? Mode.valueOf(args.get(MODE).toUpperCase(Locale.ENGLISH)) : Mode.NORMAL;
+    String userDictionaryPath = args.get(USER_DICT_PATH);
+    try {
+      if (userDictionaryPath != null) {
+        InputStream stream = loader.openResource(userDictionaryPath);
+        String encoding = args.get(USER_DICT_ENCODING);
+        if (encoding == null) {
+          encoding = IOUtils.UTF_8;
+        }
+        // note: we could allow for other encodings here as an argument
+        CharsetDecoder decoder = Charset.forName(encoding).newDecoder()
+            .onMalformedInput(CodingErrorAction.REPORT)
+            .onUnmappableCharacter(CodingErrorAction.REPORT);
+        Reader reader = new InputStreamReader(stream, decoder);
+        this.segmenter = new Segmenter(new UserDictionary(reader), mode);
+      } else {
+        this.segmenter = new Segmenter(mode);
+      }
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
+  }
+  
+  @Override
+  public Tokenizer create(Reader input) {
+    return new KuromojiTokenizer(segmenter, input);
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/StringMockSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/StringMockSolrResourceLoader.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/StringMockSolrResourceLoader.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/StringMockSolrResourceLoader.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,46 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.solr.common.ResourceLoader;
+
+class StringMockSolrResourceLoader implements ResourceLoader {
+  String text;
+
+  StringMockSolrResourceLoader(String text) {
+    this.text = text;
+  }
+
+  public List<String> getLines(String resource) throws IOException {
+    return Arrays.asList(text.split("\n"));
+  }
+
+  public Object newInstance(String cname, String... subpackages) {
+    return null;
+  }
+
+  public InputStream openResource(String resource) throws IOException {
+    return new ByteArrayInputStream(text.getBytes("UTF-8"));
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestICUCollationKeyFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestICUCollationKeyFilterFactory.java?rev=1230748&r1=1230747&r2=1230748&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestICUCollationKeyFilterFactory.java (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestICUCollationKeyFilterFactory.java Thu Jan 12 20:10:48 2012
@@ -17,18 +17,14 @@ package org.apache.solr.analysis;
  * limitations under the License.
  */
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.StringReader;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.solr.common.ResourceLoader;
 
 import com.ibm.icu.text.Collator;
 import com.ibm.icu.text.RuleBasedCollator;
@@ -263,26 +259,6 @@ public class TestICUCollationKeyFilterFa
     assertCollatesToSame(tsUmlaut, tsOE);
   }
   
-  private class StringMockSolrResourceLoader implements ResourceLoader {
-    String text;
-
-    StringMockSolrResourceLoader(String text) {
-      this.text = text;
-    }
-
-    public List<String> getLines(String resource) throws IOException {
-      return null;
-    }
-
-    public Object newInstance(String cname, String... subpackages) {
-      return null;
-    }
-
-    public InputStream openResource(String resource) throws IOException {
-      return new ByteArrayInputStream(text.getBytes("UTF-8"));
-    }
-  }
-  
   private void assertCollatesToSame(TokenStream stream1, TokenStream stream2) throws IOException {
     assertCollation(stream1, stream2, 0);
   }

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiBaseFormFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiBaseFormFilterFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiBaseFormFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiBaseFormFilterFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,41 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.StringReader;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * Simple tests for {@link KuromojiBaseFormFilterFactory}
+ */
+public class TestKuromojiBaseFormFilterFactory extends BaseTokenTestCase {
+  public void testBasics() throws IOException {
+    KuromojiTokenizerFactory tokenizerFactory = new KuromojiTokenizerFactory();
+    tokenizerFactory.init(DEFAULT_VERSION_PARAM);
+    tokenizerFactory.inform(new SolrResourceLoader(null, null));
+    TokenStream ts = tokenizerFactory.create(new StringReader("それはまだ実験段階にあります"));
+    KuromojiBaseFormFilterFactory factory = new KuromojiBaseFormFilterFactory();
+    ts = factory.create(ts);
+    assertTokenStreamContents(ts,
+        new String[] { "それ", "は", "まだ", "実験", "段階", "に", "ある", "ます"  }
+    );
+  }
+}

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiPartOfSpeechStopFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiPartOfSpeechStopFilterFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiPartOfSpeechStopFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiPartOfSpeechStopFilterFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,52 @@
+package org.apache.solr.analysis;
+
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * 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.
+ */
+
+/**
+ * Simple tests for {@link KuromojiPartOfSpeechStopFilter}
+ */
+public class TestKuromojiPartOfSpeechStopFilterFactory extends BaseTokenTestCase {
+  public void testBasics() throws IOException {
+    String tags = 
+        "#  verb-main:\n" +
+        "動詞-自立\n";
+    
+    KuromojiTokenizerFactory tokenizerFactory = new KuromojiTokenizerFactory();
+    tokenizerFactory.init(DEFAULT_VERSION_PARAM);
+    tokenizerFactory.inform(new SolrResourceLoader(null, null));
+    TokenStream ts = tokenizerFactory.create(new StringReader("私は制限スピードを超える。"));
+    KuromojiPartOfSpeechStopFilterFactory factory = new KuromojiPartOfSpeechStopFilterFactory();
+    Map<String,String> args = new HashMap<String,String>();
+    args.put("luceneMatchVersion", TEST_VERSION_CURRENT.toString());
+    args.put("tags", "stoptags.txt");
+    factory.init(args);
+    factory.inform(new StringMockSolrResourceLoader(tags));
+    ts = factory.create(ts);
+    assertTokenStreamContents(ts,
+        new String[] { "私", "は", "制限", "スピード", "を" }
+    );
+  }
+}

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiTokenizerFactory.java?rev=1230748&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiTokenizerFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestKuromojiTokenizerFactory.java Thu Jan 12 20:10:48 2012
@@ -0,0 +1,61 @@
+package org.apache.solr.analysis;
+
+/**
+ * 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.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.solr.core.SolrResourceLoader;
+
+/**
+ * Simple tests for {@link KuromojiTokenizerFactory}
+ */
+public class TestKuromojiTokenizerFactory extends BaseTokenTestCase {
+  public void testSimple() throws IOException {
+    KuromojiTokenizerFactory factory = new KuromojiTokenizerFactory();
+    factory.init(DEFAULT_VERSION_PARAM);
+    factory.inform(new SolrResourceLoader(null, null));
+    TokenStream ts = factory.create(new StringReader("これは本ではない"));
+    assertTokenStreamContents(ts,
+        new String[] { "これ", "は", "本", "で", "は", "ない" },
+        new int[] { 0, 2, 3, 4, 5, 6 },
+        new int[] { 2, 3, 4, 5, 6, 8 }
+    );
+  }
+  
+  public void testUserDict() throws IOException {
+    String userDict = 
+        "# Custom segmentation for long entries\n" +
+        "日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞\n" +
+        "関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,テスト名詞\n" +
+        "# Custom reading for sumo wrestler\n" +
+        "朝青龍,朝青龍,アサショウリュウ,カスタム人名\n";
+    KuromojiTokenizerFactory factory = new KuromojiTokenizerFactory();
+    Map<String,String> args = new HashMap<String,String>();
+    args.put("user-dictionary", "userdict.txt");
+    factory.init(args);
+    factory.inform(new StringMockSolrResourceLoader(userDict));
+    TokenStream ts = factory.create(new StringReader("関西国際空港に行った"));
+    assertTokenStreamContents(ts,
+        new String[] { "関西", "国際", "空港", "に",  "行っ",  "た" }
+    );
+  }
+}