You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by gs...@apache.org on 2008/05/16 14:22:51 UTC

svn commit: r657027 [1/2] - in /lucene/java/trunk: ./ contrib/analyzers/src/java/org/apache/lucene/analysis/compound/ contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ contrib/analyzers/src/test/org/apache/lucene/analysis/comp...

Author: gsingers
Date: Fri May 16 05:22:50 2008
New Revision: 657027

URL: http://svn.apache.org/viewvc?rev=657027&view=rev
Log:
LUCENE-1166: Added token filter for decomposing compound words

Added:
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/TernaryTree.java   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/hyphenation.dtd   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/package.html   (with props)
    lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/package.html   (with props)
    lucene/java/trunk/contrib/analyzers/src/test/org/apache/lucene/analysis/compound/
    lucene/java/trunk/contrib/analyzers/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=657027&r1=657026&r2=657027&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Fri May 16 05:22:50 2008
@@ -159,6 +159,7 @@
 12. LUCENE-400: Added word based n-gram filter (in contrib/analyzers) called ShingleFilter and an Analyzer wrapper
     that wraps another Analyzer's token stream with a ShingleFilter (Sebastian Kirsch, Steve Rowe via Grant Ingersoll) 
 
+13. LUCENE-1166: Decomposition tokenfilter for languages like German and Swedish (Thomas Peuss via Grant Ingersoll)
 
 Optimizations
 

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java Fri May 16 05:22:50 2008
@@ -0,0 +1,169 @@
+package org.apache.lucene.analysis.compound;
+
+/**
+ * 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.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Set;
+
+import org.apache.lucene.analysis.CharArraySet;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * Base class for decomposition token filters.
+ */
+public abstract class CompoundWordTokenFilterBase extends TokenFilter {
+  /**
+   * The default for minimal word length that gets decomposed
+   */
+  public static final int DEFAULT_MIN_WORD_SIZE = 5;
+
+  /**
+   * The default for minimal length of subwords that get propagated to the output of this filter
+   */
+  public static final int DEFAULT_MIN_SUBWORD_SIZE = 2;
+
+  /**
+   * The default for maximal length of subwords that get propagated to the output of this filter
+   */
+  public static final int DEFAULT_MAX_SUBWORD_SIZE = 15;
+  
+  protected final CharArraySet dictionary;
+  protected final LinkedList tokens;
+  protected final int minWordSize;
+  protected final int minSubwordSize;
+  protected final int maxSubwordSize;
+  protected final boolean onlyLongestMatch;
+
+  protected CompoundWordTokenFilterBase(TokenStream input, String[] dictionary, int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    this(input,makeDictionary(dictionary),minWordSize,minSubwordSize,maxSubwordSize, onlyLongestMatch);
+  }
+  
+  protected CompoundWordTokenFilterBase(TokenStream input, String[] dictionary, boolean onlyLongestMatch) {
+    this(input,makeDictionary(dictionary),DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, onlyLongestMatch);
+  }
+
+  protected CompoundWordTokenFilterBase(TokenStream input, Set dictionary, boolean onlyLongestMatch) {
+    this(input,dictionary,DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, onlyLongestMatch);
+  }
+
+  protected CompoundWordTokenFilterBase(TokenStream input, String[] dictionary) {
+    this(input,makeDictionary(dictionary),DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  protected CompoundWordTokenFilterBase(TokenStream input, Set dictionary) {
+    this(input,dictionary,DEFAULT_MIN_WORD_SIZE,DEFAULT_MIN_SUBWORD_SIZE,DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  protected CompoundWordTokenFilterBase(TokenStream input, Set dictionary, int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input);
+    
+    this.tokens=new LinkedList();
+    this.minWordSize=minWordSize;
+    this.minSubwordSize=minSubwordSize;
+    this.maxSubwordSize=maxSubwordSize;
+    this.onlyLongestMatch=onlyLongestMatch;
+    
+    if (dictionary instanceof CharArraySet) {
+      this.dictionary = (CharArraySet) dictionary;
+    } else {
+      this.dictionary = new CharArraySet(dictionary.size(), false);
+      addAllLowerCase(this.dictionary, dictionary);
+    }
+  }
+
+  /**
+   * Create a set of words from an array
+   * The resulting Set does case insensitive matching
+   * TODO We should look for a faster dictionary lookup approach.
+   * @param dictionary
+   * @return
+   */
+  public static final Set makeDictionary(final String[] dictionary) {
+    CharArraySet dict = new CharArraySet(dictionary.length, false);
+    addAllLowerCase(dict, Arrays.asList(dictionary));
+    return dict;
+  }
+  
+  public Token next() throws IOException {
+    if (tokens.size() > 0) {
+      return (Token)tokens.removeFirst();
+    }
+
+    Token token = input.next();
+    if (token == null) {
+      return null;
+    }
+
+    decompose(token);
+
+    if (tokens.size() > 0) {
+      return (Token)tokens.removeFirst();
+    } else {
+      return null;
+    }
+  }
+  
+  protected static final void addAllLowerCase(Set target, Collection col) {
+    Iterator iter=col.iterator();
+    
+    while (iter.hasNext()) {
+      target.add(((String)iter.next()).toLowerCase());
+    }
+  }
+  
+  protected static char[] makeLowerCaseCopy(final char[] buffer) {
+    char[] result=new char[buffer.length];
+    System.arraycopy(buffer, 0, result, 0, buffer.length);
+    
+    for (int i=0;i<buffer.length;++i) {
+       result[i]=Character.toLowerCase(buffer[i]);
+    }
+    
+    return result;
+  }
+  
+  protected final Token createToken(final int offset, final int length,
+      final Token prototype) {
+    Token t = new Token(prototype.startOffset() + offset, prototype
+        .startOffset()
+        + offset + length, prototype.type());
+    t.setTermBuffer(prototype.termBuffer(), offset, length);
+    t.setPositionIncrement(0);
+    return t;
+  }
+
+  protected void decompose(final Token token) {
+    // In any case we give the original token back
+    tokens.add(token);
+
+    // Only words longer than minWordSize get processed
+    if (token.termLength() < this.minWordSize) {
+      return;
+    }
+    
+    decomposeInternal(token);
+  }
+  
+  protected abstract void decomposeInternal(final Token token);
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java Fri May 16 05:22:50 2008
@@ -0,0 +1,114 @@
+package org.apache.lucene.analysis.compound;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * A TokenFilter that decomposes compound words found in many germanic languages
+ * "Donaudampfschiff" becomes Donau, dampf, schiff so that you can find
+ * "Donaudampfschiff" even when you only enter "schiff". 
+ *  It uses a brute-force algorithm to achieve this.
+ */
+public class DictionaryCompoundWordTokenFilter extends CompoundWordTokenFilterBase {
+  /**
+   * 
+   * @param input the token stream to process
+   * @param dictionary the word dictionary to match against
+   * @param minWordSize only words longer than this get processed
+   * @param minSubwordSize only subwords longer than this get to the output stream
+   * @param maxSubwordSize only subwords shorter than this get to the output stream
+   * @param onlyLongestMatch Add only the longest matching subword to the stream
+   */
+  public DictionaryCompoundWordTokenFilter(TokenStream input, String[] dictionary,
+      int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input, dictionary, minWordSize, minSubwordSize, maxSubwordSize, onlyLongestMatch);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param dictionary the word dictionary to match against
+   */
+  public DictionaryCompoundWordTokenFilter(TokenStream input, String[] dictionary) {
+    super(input, dictionary);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param dictionary the word dictionary to match against. If this is a {@link org.apache.lucene.analysis.CharArraySet CharArraySet} it must have set ignoreCase=false and only contain
+   *        lower case strings. 
+   */
+  public DictionaryCompoundWordTokenFilter(TokenStream input, Set dictionary) {
+    super(input, dictionary);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param dictionary the word dictionary to match against. If this is a {@link org.apache.lucene.analysis.CharArraySet CharArraySet} it must have set ignoreCase=false and only contain
+   *        lower case strings. 
+   * @param minWordSize only words longer than this get processed
+   * @param minSubwordSize only subwords longer than this get to the output stream
+   * @param maxSubwordSize only subwords shorter than this get to the output stream
+   * @param onlyLongestMatch Add only the longest matching subword to the stream
+   */
+  public DictionaryCompoundWordTokenFilter(TokenStream input, Set dictionary,
+      int minWordSize, int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input, dictionary, minWordSize, minSubwordSize, maxSubwordSize, onlyLongestMatch);
+  }
+
+  protected void decomposeInternal(final Token token) {
+    // Only words longer than minWordSize get processed
+    if (token.termLength() < this.minWordSize) {
+      return;
+    }
+    
+    char[] lowerCaseTermBuffer=makeLowerCaseCopy(token.termBuffer());
+    
+    for (int i=0;i<token.termLength()-this.minSubwordSize;++i) {
+        Token longestMatchToken=null;
+        for (int j=this.minSubwordSize-1;j<this.maxSubwordSize;++j) {
+            if(i+j>token.termLength()) {
+                break;
+            }
+            if(dictionary.contains(lowerCaseTermBuffer, i, j)) {
+                if (this.onlyLongestMatch) {
+                   if (longestMatchToken!=null) {
+                     if (longestMatchToken.termLength()<j) {
+                       longestMatchToken=createToken(i,j,token);
+                     }
+                   } else {
+                     longestMatchToken=createToken(i,j,token);
+                   }
+                } else {
+                   tokens.add(createToken(i,j,token));
+                }
+            } 
+        }
+        if (this.onlyLongestMatch && longestMatchToken!=null) {
+          tokens.add(longestMatchToken);
+        }
+    }
+  }
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/DictionaryCompoundWordTokenFilter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java Fri May 16 05:22:50 2008
@@ -0,0 +1,217 @@
+package org.apache.lucene.analysis.compound;
+
+/**
+ * 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.InputStreamReader;
+import java.io.Reader;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.compound.hyphenation.Hyphenation;
+import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
+import org.xml.sax.InputSource;
+
+/**
+ * A TokenFilter that decomposes compound words found in many germanic languages
+ * "Donaudampfschiff" becomes Donau, dampf, schiff so that you can find
+ * "Donaudampfschiff" even when you only enter "schiff" It uses a hyphenation
+ * grammar and a word dictionary to achieve this.
+ */
+public class HyphenationCompoundWordTokenFilter extends
+    CompoundWordTokenFilterBase {
+  private HyphenationTree hyphenator;
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param hyphenator the hyphenation pattern tree to use for hyphenation
+   * @param dictionary the word dictionary to match against
+   * @param minWordSize only words longer than this get processed
+   * @param minSubwordSize only subwords longer than this get to the output
+   *        stream
+   * @param maxSubwordSize only subwords shorter than this get to the output
+   *        stream
+   * @param onlyLongestMatch Add only the longest matching subword to the stream
+   */
+  public HyphenationCompoundWordTokenFilter(TokenStream input,
+      HyphenationTree hyphenator, String[] dictionary, int minWordSize,
+      int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    this(input, hyphenator, makeDictionary(dictionary), minWordSize,
+        minSubwordSize, maxSubwordSize, onlyLongestMatch);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param hyphenator the hyphenation pattern tree to use for hyphenation
+   * @param dictionary the word dictionary to match against
+   */
+  public HyphenationCompoundWordTokenFilter(TokenStream input,
+      HyphenationTree hyphenator, String[] dictionary) {
+    this(input, hyphenator, makeDictionary(dictionary), DEFAULT_MIN_WORD_SIZE,
+        DEFAULT_MIN_SUBWORD_SIZE, DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param hyphenator the hyphenation pattern tree to use for hyphenation
+   * @param dictionary the word dictionary to match against. If this is a {@link org.apache.lucene.analysis.CharArraySet CharArraySet} it must have set ignoreCase=false and only contain
+   *        lower case strings. 
+   */
+  public HyphenationCompoundWordTokenFilter(TokenStream input,
+      HyphenationTree hyphenator, Set dictionary) {
+    this(input, hyphenator, dictionary, DEFAULT_MIN_WORD_SIZE,
+        DEFAULT_MIN_SUBWORD_SIZE, DEFAULT_MAX_SUBWORD_SIZE, false);
+  }
+
+  /**
+   * 
+   * @param input the token stream to process
+   * @param hyphenator the hyphenation pattern tree to use for hyphenation
+   * @param dictionary the word dictionary to match against. If this is a {@link org.apache.lucene.analysis.CharArraySet CharArraySet} it must have set ignoreCase=false and only contain
+   *        lower case strings. 
+   * @param minWordSize only words longer than this get processed
+   * @param minSubwordSize only subwords longer than this get to the output
+   *        stream
+   * @param maxSubwordSize only subwords shorter than this get to the output
+   *        stream
+   * @param onlyLongestMatch Add only the longest matching subword to the stream
+   */
+  public HyphenationCompoundWordTokenFilter(TokenStream input,
+      HyphenationTree hyphenator, Set dictionary, int minWordSize,
+      int minSubwordSize, int maxSubwordSize, boolean onlyLongestMatch) {
+    super(input, dictionary, minWordSize, minSubwordSize, maxSubwordSize,
+        onlyLongestMatch);
+
+    this.hyphenator = hyphenator;
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationFilename the filename of the XML grammar to load
+   * @return An object representing the hyphenation patterns
+   * @throws Exception
+   */
+  public static HyphenationTree getHyphenationTree(String hyphenationFilename)
+      throws Exception {
+    return getHyphenationTree(new File(hyphenationFilename));
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationFile the file of the XML grammar to load
+   * @return An object representing the hyphenation patterns
+   * @throws Exception
+   */
+  public static HyphenationTree getHyphenationTree(File hyphenationFile)
+      throws Exception {
+    return getHyphenationTree(new InputStreamReader(new FileInputStream(
+        hyphenationFile), "ISO-8859-1"));
+  }
+
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationReader the reader of the XML grammar to load from
+   * @return An object representing the hyphenation patterns
+   * @throws Exception
+   */
+  public static HyphenationTree getHyphenationTree(Reader hyphenationReader)
+      throws Exception {
+    HyphenationTree tree = new HyphenationTree();
+
+    tree.loadPatterns(new InputSource(hyphenationReader));
+
+    return tree;
+  }
+
+  protected void decomposeInternal(final Token token) {
+    // get the hpyphenation points
+    Hyphenation hyphens = hyphenator.hyphenate(token.termBuffer(), 0, token
+        .termLength(), 1, 1);
+    // No hyphen points found -> exit
+    if (hyphens == null) {
+      return;
+    }
+
+    final int[] hyp = hyphens.getHyphenationPoints();
+    char[] lowerCaseTermBuffer=makeLowerCaseCopy(token.termBuffer());
+
+    for (int i = 0; i < hyp.length; ++i) {
+      int remaining = hyp.length - i;
+      int start = hyp[i];
+      Token longestMatchToken = null;
+      for (int j = 1; j < remaining; j++) {
+        int partLength = hyp[i + j] - start;
+
+        // if the part is longer than maxSubwordSize we
+        // are done with this round
+        if (partLength > this.maxSubwordSize) {
+          break;
+        }
+
+        // we only put subwords to the token stream
+        // that are longer than minPartSize
+        if (partLength < this.minSubwordSize) {
+          continue;
+        }
+
+        // check the dictionary
+        if (dictionary.contains(lowerCaseTermBuffer, start, partLength)) {
+          if (this.onlyLongestMatch) {
+            if (longestMatchToken != null) {
+              if (longestMatchToken.termLength() < partLength) {
+                longestMatchToken = createToken(start, partLength, token);
+              }
+            } else {
+              longestMatchToken = createToken(start, partLength, token);
+            }
+          } else {
+            tokens.add(createToken(start, partLength, token));
+          }
+        } else if (dictionary.contains(lowerCaseTermBuffer, start,
+            partLength - 1)) {
+          // check the dictionary again with a word that is one character
+          // shorter
+          // to avoid problems with genitive 's characters and other binding
+          // characters
+          if (this.onlyLongestMatch) {
+            if (longestMatchToken != null) {
+              if (longestMatchToken.termLength() < partLength - 1) {
+                longestMatchToken = createToken(start, partLength - 1, token);
+              }
+            } else {
+              longestMatchToken = createToken(start, partLength - 1, token);
+            }
+          } else {
+            tokens.add(createToken(start, partLength - 1, token));
+          }
+        }
+      }
+      if (this.onlyLongestMatch && longestMatchToken!=null) {
+        tokens.add(longestMatchToken);
+      }
+    }
+  }
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java Fri May 16 05:22:50 2008
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+import java.io.Serializable;
+
+/**
+ * This class implements a simple byte vector with access to the underlying
+ * array.
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+public class ByteVector implements Serializable {
+
+  /**
+   * Capacity increment size
+   */
+  private static final int DEFAULT_BLOCK_SIZE = 2048;
+
+  private int blockSize;
+
+  /**
+   * The encapsulated array
+   */
+  private byte[] array;
+
+  /**
+   * Points to next free item
+   */
+  private int n;
+
+  public ByteVector() {
+    this(DEFAULT_BLOCK_SIZE);
+  }
+
+  public ByteVector(int capacity) {
+    if (capacity > 0) {
+      blockSize = capacity;
+    } else {
+      blockSize = DEFAULT_BLOCK_SIZE;
+    }
+    array = new byte[blockSize];
+    n = 0;
+  }
+
+  public ByteVector(byte[] a) {
+    blockSize = DEFAULT_BLOCK_SIZE;
+    array = a;
+    n = 0;
+  }
+
+  public ByteVector(byte[] a, int capacity) {
+    if (capacity > 0) {
+      blockSize = capacity;
+    } else {
+      blockSize = DEFAULT_BLOCK_SIZE;
+    }
+    array = a;
+    n = 0;
+  }
+
+  public byte[] getArray() {
+    return array;
+  }
+
+  /**
+   * return number of items in array
+   */
+  public int length() {
+    return n;
+  }
+
+  /**
+   * returns current capacity of array
+   */
+  public int capacity() {
+    return array.length;
+  }
+
+  public void put(int index, byte val) {
+    array[index] = val;
+  }
+
+  public byte get(int index) {
+    return array[index];
+  }
+
+  /**
+   * This is to implement memory allocation in the array. Like malloc().
+   */
+  public int alloc(int size) {
+    int index = n;
+    int len = array.length;
+    if (n + size >= len) {
+      byte[] aux = new byte[len + blockSize];
+      System.arraycopy(array, 0, aux, 0, len);
+      array = aux;
+    }
+    n += size;
+    return index;
+  }
+
+  public void trimToSize() {
+    if (n < array.length) {
+      byte[] aux = new byte[n];
+      System.arraycopy(array, 0, aux, 0, n);
+      array = aux;
+    }
+  }
+
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/ByteVector.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java Fri May 16 05:22:50 2008
@@ -0,0 +1,136 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+import java.io.Serializable;
+
+/**
+ * This class implements a simple char vector with access to the underlying
+ * array.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+public class CharVector implements Cloneable, Serializable {
+
+  /**
+   * Capacity increment size
+   */
+  private static final int DEFAULT_BLOCK_SIZE = 2048;
+
+  private int blockSize;
+
+  /**
+   * The encapsulated array
+   */
+  private char[] array;
+
+  /**
+   * Points to next free item
+   */
+  private int n;
+
+  public CharVector() {
+    this(DEFAULT_BLOCK_SIZE);
+  }
+
+  public CharVector(int capacity) {
+    if (capacity > 0) {
+      blockSize = capacity;
+    } else {
+      blockSize = DEFAULT_BLOCK_SIZE;
+    }
+    array = new char[blockSize];
+    n = 0;
+  }
+
+  public CharVector(char[] a) {
+    blockSize = DEFAULT_BLOCK_SIZE;
+    array = a;
+    n = a.length;
+  }
+
+  public CharVector(char[] a, int capacity) {
+    if (capacity > 0) {
+      blockSize = capacity;
+    } else {
+      blockSize = DEFAULT_BLOCK_SIZE;
+    }
+    array = a;
+    n = a.length;
+  }
+
+  /**
+   * Reset Vector but don't resize or clear elements
+   */
+  public void clear() {
+    n = 0;
+  }
+
+  public Object clone() {
+    CharVector cv = new CharVector((char[]) array.clone(), blockSize);
+    cv.n = this.n;
+    return cv;
+  }
+
+  public char[] getArray() {
+    return array;
+  }
+
+  /**
+   * return number of items in array
+   */
+  public int length() {
+    return n;
+  }
+
+  /**
+   * returns current capacity of array
+   */
+  public int capacity() {
+    return array.length;
+  }
+
+  public void put(int index, char val) {
+    array[index] = val;
+  }
+
+  public char get(int index) {
+    return array[index];
+  }
+
+  public int alloc(int size) {
+    int index = n;
+    int len = array.length;
+    if (n + size >= len) {
+      char[] aux = new char[len + blockSize];
+      System.arraycopy(array, 0, aux, 0, len);
+      array = aux;
+    }
+    n += size;
+    return index;
+  }
+
+  public void trimToSize() {
+    if (n < array.length) {
+      char[] aux = new char[n];
+      System.arraycopy(array, 0, aux, 0, n);
+      array = aux;
+    }
+  }
+
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/CharVector.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java Fri May 16 05:22:50 2008
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+import java.io.Serializable;
+
+/**
+ * This class represents a hyphen. A 'full' hyphen is made of 3 parts: the
+ * pre-break text, post-break text and no-break. If no line-break is generated
+ * at this position, the no-break text is used, otherwise, pre-break and
+ * post-break are used. Typically, pre-break is equal to the hyphen character
+ * and the others are empty. However, this general scheme allows support for
+ * cases in some languages where words change spelling if they're split across
+ * lines, like german's 'backen' which hyphenates 'bak-ken'. BTW, this comes
+ * from TeX.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+
+public class Hyphen implements Serializable {
+  public String preBreak;
+
+  public String noBreak;
+
+  public String postBreak;
+
+  Hyphen(String pre, String no, String post) {
+    preBreak = pre;
+    noBreak = no;
+    postBreak = post;
+  }
+
+  Hyphen(String pre) {
+    preBreak = pre;
+    noBreak = null;
+    postBreak = null;
+  }
+
+  public String toString() {
+    if (noBreak == null && postBreak == null && preBreak != null
+        && preBreak.equals("-")) {
+      return "-";
+    }
+    StringBuffer res = new StringBuffer("{");
+    res.append(preBreak);
+    res.append("}{");
+    res.append(postBreak);
+    res.append("}{");
+    res.append(noBreak);
+    res.append('}');
+    return res.toString();
+  }
+
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphen.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java Fri May 16 05:22:50 2008
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+/**
+ * This class represents a hyphenated word.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified.
+ */
+public class Hyphenation {
+
+  private int[] hyphenPoints;
+
+  /**
+   * number of hyphenation points in word
+   */
+  private int len;
+
+  /**
+   * rawWord as made of alternating strings and {@link Hyphen Hyphen} instances
+   */
+  Hyphenation(int[] points) {
+    hyphenPoints = points;
+  }
+
+  /**
+   * @return the number of hyphenation points in the word
+   */
+  public int length() {
+    return hyphenPoints.length;
+  }
+
+  /**
+   * @return the hyphenation points
+   */
+  public int[] getHyphenationPoints() {
+    return hyphenPoints;
+  }
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/Hyphenation.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java Fri May 16 05:22:50 2008
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+/**
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+public class HyphenationException extends Exception {
+
+  /**
+   * @see java.lang.Throwable#Throwable(String)
+   */
+  public HyphenationException(String msg) {
+    super(msg);
+  }
+
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationException.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java Fri May 16 05:22:50 2008
@@ -0,0 +1,475 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+import java.io.File;
+import java.io.Serializable;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import org.xml.sax.InputSource;
+
+/**
+ * This tree structure stores the hyphenation patterns in an efficient way for
+ * fast lookup. It provides the provides the method to hyphenate a word.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+public class HyphenationTree extends TernaryTree implements PatternConsumer,
+    Serializable {
+
+  private static final long serialVersionUID = -7842107987915665573L;
+
+  /**
+   * value space: stores the interletter values
+   */
+  protected ByteVector vspace;
+
+  /**
+   * This map stores hyphenation exceptions
+   */
+  protected HashMap stoplist;
+
+  /**
+   * This map stores the character classes
+   */
+  protected TernaryTree classmap;
+
+  /**
+   * Temporary map to store interletter values on pattern loading.
+   */
+  private transient TernaryTree ivalues;
+
+  public HyphenationTree() {
+    stoplist = new HashMap(23); // usually a small table
+    classmap = new TernaryTree();
+    vspace = new ByteVector();
+    vspace.alloc(1); // this reserves index 0, which we don't use
+  }
+
+  /**
+   * Packs the values by storing them in 4 bits, two values into a byte Values
+   * range is from 0 to 9. We use zero as terminator, so we'll add 1 to the
+   * value.
+   * 
+   * @param values a string of digits from '0' to '9' representing the
+   *        interletter values.
+   * @return the index into the vspace array where the packed values are stored.
+   */
+  protected int packValues(String values) {
+    int i, n = values.length();
+    int m = (n & 1) == 1 ? (n >> 1) + 2 : (n >> 1) + 1;
+    int offset = vspace.alloc(m);
+    byte[] va = vspace.getArray();
+    for (i = 0; i < n; i++) {
+      int j = i >> 1;
+      byte v = (byte) ((values.charAt(i) - '0' + 1) & 0x0f);
+      if ((i & 1) == 1) {
+        va[j + offset] = (byte) (va[j + offset] | v);
+      } else {
+        va[j + offset] = (byte) (v << 4); // big endian
+      }
+    }
+    va[m - 1 + offset] = 0; // terminator
+    return offset;
+  }
+
+  protected String unpackValues(int k) {
+    StringBuffer buf = new StringBuffer();
+    byte v = vspace.get(k++);
+    while (v != 0) {
+      char c = (char) ((v >>> 4) - 1 + '0');
+      buf.append(c);
+      c = (char) (v & 0x0f);
+      if (c == 0) {
+        break;
+      }
+      c = (char) (c - 1 + '0');
+      buf.append(c);
+      v = vspace.get(k++);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * Read hyphenation patterns from an XML file.
+   * 
+   * @param filename the filename
+   * @throws HyphenationException In case the parsing fails
+   */
+  public void loadPatterns(File f) throws HyphenationException {
+    try {
+      InputSource src = new InputSource(f.toURL().toExternalForm());
+      loadPatterns(src);
+    } catch (MalformedURLException e) {
+      throw new HyphenationException("Error converting the File '" + f
+          + "' to a URL: " + e.getMessage());
+    }
+  }
+
+  /**
+   * Read hyphenation patterns from an XML file.
+   * 
+   * @param source the InputSource for the file
+   * @throws HyphenationException In case the parsing fails
+   */
+  public void loadPatterns(InputSource source) throws HyphenationException {
+    PatternParser pp = new PatternParser(this);
+    ivalues = new TernaryTree();
+
+    pp.parse(source);
+
+    // patterns/values should be now in the tree
+    // let's optimize a bit
+    trimToSize();
+    vspace.trimToSize();
+    classmap.trimToSize();
+
+    // get rid of the auxiliary map
+    ivalues = null;
+  }
+
+  public String findPattern(String pat) {
+    int k = super.find(pat);
+    if (k >= 0) {
+      return unpackValues(k);
+    }
+    return "";
+  }
+
+  /**
+   * String compare, returns 0 if equal or t is a substring of s
+   */
+  protected int hstrcmp(char[] s, int si, char[] t, int ti) {
+    for (; s[si] == t[ti]; si++, ti++) {
+      if (s[si] == 0) {
+        return 0;
+      }
+    }
+    if (t[ti] == 0) {
+      return 0;
+    }
+    return s[si] - t[ti];
+  }
+
+  protected byte[] getValues(int k) {
+    StringBuffer buf = new StringBuffer();
+    byte v = vspace.get(k++);
+    while (v != 0) {
+      char c = (char) ((v >>> 4) - 1);
+      buf.append(c);
+      c = (char) (v & 0x0f);
+      if (c == 0) {
+        break;
+      }
+      c = (char) (c - 1);
+      buf.append(c);
+      v = vspace.get(k++);
+    }
+    byte[] res = new byte[buf.length()];
+    for (int i = 0; i < res.length; i++) {
+      res[i] = (byte) buf.charAt(i);
+    }
+    return res;
+  }
+
+  /**
+   * <p>
+   * Search for all possible partial matches of word starting at index an update
+   * interletter values. In other words, it does something like:
+   * </p>
+   * <code>
+   * for(i=0; i<patterns.length; i++) {
+   * if ( word.substring(index).startsWidth(patterns[i]) )
+   * update_interletter_values(patterns[i]);
+   * }
+   * </code>
+   * <p>
+   * But it is done in an efficient way since the patterns are stored in a
+   * ternary tree. In fact, this is the whole purpose of having the tree: doing
+   * this search without having to test every single pattern. The number of
+   * patterns for languages such as English range from 4000 to 10000. Thus,
+   * doing thousands of string comparisons for each word to hyphenate would be
+   * really slow without the tree. The tradeoff is memory, but using a ternary
+   * tree instead of a trie, almost halves the the memory used by Lout or TeX.
+   * It's also faster than using a hash table
+   * </p>
+   * 
+   * @param word null terminated word to match
+   * @param index start index from word
+   * @param il interletter values array to update
+   */
+  protected void searchPatterns(char[] word, int index, byte[] il) {
+    byte[] values;
+    int i = index;
+    char p, q;
+    char sp = word[i];
+    p = root;
+
+    while (p > 0 && p < sc.length) {
+      if (sc[p] == 0xFFFF) {
+        if (hstrcmp(word, i, kv.getArray(), lo[p]) == 0) {
+          values = getValues(eq[p]); // data pointer is in eq[]
+          int j = index;
+          for (int k = 0; k < values.length; k++) {
+            if (j < il.length && values[k] > il[j]) {
+              il[j] = values[k];
+            }
+            j++;
+          }
+        }
+        return;
+      }
+      int d = sp - sc[p];
+      if (d == 0) {
+        if (sp == 0) {
+          break;
+        }
+        sp = word[++i];
+        p = eq[p];
+        q = p;
+
+        // look for a pattern ending at this position by searching for
+        // the null char ( splitchar == 0 )
+        while (q > 0 && q < sc.length) {
+          if (sc[q] == 0xFFFF) { // stop at compressed branch
+            break;
+          }
+          if (sc[q] == 0) {
+            values = getValues(eq[q]);
+            int j = index;
+            for (int k = 0; k < values.length; k++) {
+              if (j < il.length && values[k] > il[j]) {
+                il[j] = values[k];
+              }
+              j++;
+            }
+            break;
+          } else {
+            q = lo[q];
+
+            /**
+             * actually the code should be: q = sc[q] < 0 ? hi[q] : lo[q]; but
+             * java chars are unsigned
+             */
+          }
+        }
+      } else {
+        p = d < 0 ? lo[p] : hi[p];
+      }
+    }
+  }
+
+  /**
+   * Hyphenate word and return a Hyphenation object.
+   * 
+   * @param word the word to be hyphenated
+   * @param remainCharCount Minimum number of characters allowed before the
+   *        hyphenation point.
+   * @param pushCharCount Minimum number of characters allowed after the
+   *        hyphenation point.
+   * @return a {@link Hyphenation Hyphenation} object representing the
+   *         hyphenated word or null if word is not hyphenated.
+   */
+  public Hyphenation hyphenate(String word, int remainCharCount,
+      int pushCharCount) {
+    char[] w = word.toCharArray();
+    return hyphenate(w, 0, w.length, remainCharCount, pushCharCount);
+  }
+
+  /**
+   * w = "****nnllllllnnn*****", where n is a non-letter, l is a letter, all n
+   * may be absent, the first n is at offset, the first l is at offset +
+   * iIgnoreAtBeginning; word = ".llllll.'\0'***", where all l in w are copied
+   * into word. In the first part of the routine len = w.length, in the second
+   * part of the routine len = word.length. Three indices are used: index(w),
+   * the index in w, index(word), the index in word, letterindex(word), the
+   * index in the letter part of word. The following relations exist: index(w) =
+   * offset + i - 1 index(word) = i - iIgnoreAtBeginning letterindex(word) =
+   * index(word) - 1 (see first loop). It follows that: index(w) - index(word) =
+   * offset - 1 + iIgnoreAtBeginning index(w) = letterindex(word) + offset +
+   * iIgnoreAtBeginning
+   */
+
+  /**
+   * Hyphenate word and return an array of hyphenation points.
+   * 
+   * @param w char array that contains the word
+   * @param offset Offset to first character in word
+   * @param len Length of word
+   * @param remainCharCount Minimum number of characters allowed before the
+   *        hyphenation point.
+   * @param pushCharCount Minimum number of characters allowed after the
+   *        hyphenation point.
+   * @return a {@link Hyphenation Hyphenation} object representing the
+   *         hyphenated word or null if word is not hyphenated.
+   */
+  public Hyphenation hyphenate(char[] w, int offset, int len,
+      int remainCharCount, int pushCharCount) {
+    int i;
+    char[] word = new char[len + 3];
+
+    // normalize word
+    char[] c = new char[2];
+    int iIgnoreAtBeginning = 0;
+    int iLength = len;
+    boolean bEndOfLetters = false;
+    for (i = 1; i <= len; i++) {
+      c[0] = w[offset + i - 1];
+      int nc = classmap.find(c, 0);
+      if (nc < 0) { // found a non-letter character ...
+        if (i == (1 + iIgnoreAtBeginning)) {
+          // ... before any letter character
+          iIgnoreAtBeginning++;
+        } else {
+          // ... after a letter character
+          bEndOfLetters = true;
+        }
+        iLength--;
+      } else {
+        if (!bEndOfLetters) {
+          word[i - iIgnoreAtBeginning] = (char) nc;
+        } else {
+          return null;
+        }
+      }
+    }
+    len = iLength;
+    if (len < (remainCharCount + pushCharCount)) {
+      // word is too short to be hyphenated
+      return null;
+    }
+    int[] result = new int[len + 1];
+    int k = 0;
+
+    // check exception list first
+    String sw = new String(word, 1, len);
+    if (stoplist.containsKey(sw)) {
+      // assume only simple hyphens (Hyphen.pre="-", Hyphen.post = Hyphen.no =
+      // null)
+      ArrayList hw = (ArrayList) stoplist.get(sw);
+      int j = 0;
+      for (i = 0; i < hw.size(); i++) {
+        Object o = hw.get(i);
+        // j = index(sw) = letterindex(word)?
+        // result[k] = corresponding index(w)
+        if (o instanceof String) {
+          j += ((String) o).length();
+          if (j >= remainCharCount && j < (len - pushCharCount)) {
+            result[k++] = j + iIgnoreAtBeginning;
+          }
+        }
+      }
+    } else {
+      // use algorithm to get hyphenation points
+      word[0] = '.'; // word start marker
+      word[len + 1] = '.'; // word end marker
+      word[len + 2] = 0; // null terminated
+      byte[] il = new byte[len + 3]; // initialized to zero
+      for (i = 0; i < len + 1; i++) {
+        searchPatterns(word, i, il);
+      }
+
+      // hyphenation points are located where interletter value is odd
+      // i is letterindex(word),
+      // i + 1 is index(word),
+      // result[k] = corresponding index(w)
+      for (i = 0; i < len; i++) {
+        if (((il[i + 1] & 1) == 1) && i >= remainCharCount
+            && i <= (len - pushCharCount)) {
+          result[k++] = i + iIgnoreAtBeginning;
+        }
+      }
+    }
+
+    if (k > 0) {
+      // trim result array
+      int[] res = new int[k+2];
+      System.arraycopy(result, 0, res, 1, k);
+      // We add the synthetical hyphenation points
+      // at the beginning and end of the word
+      res[0]=0;
+      res[k+1]=len;
+      return new Hyphenation(res);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Add a character class to the tree. It is used by
+   * {@link PatternParser PatternParser} as callback to add character classes.
+   * Character classes define the valid word characters for hyphenation. If a
+   * word contains a character not defined in any of the classes, it is not
+   * hyphenated. It also defines a way to normalize the characters in order to
+   * compare them with the stored patterns. Usually pattern files use only lower
+   * case characters, in this case a class for letter 'a', for example, should
+   * be defined as "aA", the first character being the normalization char.
+   */
+  public void addClass(String chargroup) {
+    if (chargroup.length() > 0) {
+      char equivChar = chargroup.charAt(0);
+      char[] key = new char[2];
+      key[1] = 0;
+      for (int i = 0; i < chargroup.length(); i++) {
+        key[0] = chargroup.charAt(i);
+        classmap.insert(key, 0, equivChar);
+      }
+    }
+  }
+
+  /**
+   * Add an exception to the tree. It is used by
+   * {@link PatternParser PatternParser} class as callback to store the
+   * hyphenation exceptions.
+   * 
+   * @param word normalized word
+   * @param hyphenatedword a vector of alternating strings and
+   *        {@link Hyphen hyphen} objects.
+   */
+  public void addException(String word, ArrayList hyphenatedword) {
+    stoplist.put(word, hyphenatedword);
+  }
+
+  /**
+   * Add a pattern to the tree. Mainly, to be used by
+   * {@link PatternParser PatternParser} class as callback to add a pattern to
+   * the tree.
+   * 
+   * @param pattern the hyphenation pattern
+   * @param ivalue interletter weight values indicating the desirability and
+   *        priority of hyphenating at a given point within the pattern. It
+   *        should contain only digit characters. (i.e. '0' to '9').
+   */
+  public void addPattern(String pattern, String ivalue) {
+    int k = ivalues.find(ivalue);
+    if (k <= 0) {
+      k = packValues(ivalue);
+      ivalues.insert(ivalue, (char) k);
+    }
+    insert(pattern, (char) k);
+  }
+
+  public void printStats() {
+    System.out.println("Value space size = "
+        + Integer.toString(vspace.length()));
+    super.printStats();
+
+  }
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java Fri May 16 05:22:50 2008
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+import java.util.ArrayList;
+
+/**
+ * This interface is used to connect the XML pattern file parser to the
+ * hyphenation tree.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified.
+ */
+public interface PatternConsumer {
+
+  /**
+   * Add a character class. A character class defines characters that are
+   * considered equivalent for the purpose of hyphenation (e.g. "aA"). It
+   * usually means to ignore case.
+   * 
+   * @param chargroup character group
+   */
+  void addClass(String chargroup);
+
+  /**
+   * Add a hyphenation exception. An exception replaces the result obtained by
+   * the algorithm for cases for which this fails or the user wants to provide
+   * his own hyphenation. A hyphenatedword is a vector of alternating String's
+   * and {@link Hyphen Hyphen} instances
+   */
+  void addException(String word, ArrayList hyphenatedword);
+
+  /**
+   * Add hyphenation patterns.
+   * 
+   * @param pattern the pattern
+   * @param values interletter values expressed as a string of digit characters.
+   */
+  void addPattern(String pattern, String values);
+
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternConsumer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java?rev=657027&view=auto
==============================================================================
--- lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java (added)
+++ lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java Fri May 16 05:22:50 2008
@@ -0,0 +1,518 @@
+/*
+ * 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.
+ */
+
+/* $Id: PatternParser.java 426576 2006-07-28 15:44:37Z jeremias $ */
+
+package org.apache.lucene.analysis.compound.hyphenation;
+
+// SAX
+import org.xml.sax.XMLReader;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.SAXParseException;
+import org.xml.sax.helpers.DefaultHandler;
+import org.xml.sax.Attributes;
+
+// Java
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.StringReader;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+
+import javax.xml.parsers.SAXParserFactory;
+
+/**
+ * A SAX document handler to read and parse hyphenation patterns from a XML
+ * file.
+ * 
+ * This class has been taken from the Apache FOP project (http://xmlgraphics.apache.org/fop/). They have been slightly modified. 
+ */
+public class PatternParser extends DefaultHandler implements PatternConsumer {
+
+  XMLReader parser;
+
+  int currElement;
+
+  PatternConsumer consumer;
+
+  StringBuffer token;
+
+  ArrayList exception;
+
+  char hyphenChar;
+
+  String errMsg;
+
+  static final int ELEM_CLASSES = 1;
+
+  static final int ELEM_EXCEPTIONS = 2;
+
+  static final int ELEM_PATTERNS = 3;
+
+  static final int ELEM_HYPHEN = 4;
+
+  public PatternParser() throws HyphenationException {
+    token = new StringBuffer();
+    parser = createParser();
+    parser.setContentHandler(this);
+    parser.setErrorHandler(this);
+    parser.setEntityResolver(this);
+    hyphenChar = '-'; // default
+
+  }
+
+  public PatternParser(PatternConsumer consumer) throws HyphenationException {
+    this();
+    this.consumer = consumer;
+  }
+
+  public void setConsumer(PatternConsumer consumer) {
+    this.consumer = consumer;
+  }
+
+  /**
+   * Parses a hyphenation pattern file.
+   * 
+   * @param filename the filename
+   * @throws HyphenationException In case of an exception while parsing
+   */
+  public void parse(String filename) throws HyphenationException {
+    parse(new File(filename));
+  }
+
+  /**
+   * Parses a hyphenation pattern file.
+   * 
+   * @param file the pattern file
+   * @throws HyphenationException In case of an exception while parsing
+   */
+  public void parse(File file) throws HyphenationException {
+    try {
+      InputSource src = new InputSource(file.toURL().toExternalForm());
+      parse(src);
+    } catch (MalformedURLException e) {
+      throw new HyphenationException("Error converting the File '" + file
+          + "' to a URL: " + e.getMessage());
+    }
+  }
+
+  /**
+   * Parses a hyphenation pattern file.
+   * 
+   * @param source the InputSource for the file
+   * @throws HyphenationException In case of an exception while parsing
+   */
+  public void parse(InputSource source) throws HyphenationException {
+    try {
+      parser.parse(source);
+    } catch (FileNotFoundException fnfe) {
+      throw new HyphenationException("File not found: " + fnfe.getMessage());
+    } catch (IOException ioe) {
+      throw new HyphenationException(ioe.getMessage());
+    } catch (SAXException e) {
+      throw new HyphenationException(errMsg);
+    }
+  }
+
+  /**
+   * Creates a SAX parser using JAXP
+   * 
+   * @return the created SAX parser
+   */
+  static XMLReader createParser() {
+    try {
+      SAXParserFactory factory = SAXParserFactory.newInstance();
+      factory.setNamespaceAware(true);
+      return factory.newSAXParser().getXMLReader();
+    } catch (Exception e) {
+      throw new RuntimeException("Couldn't create XMLReader: " + e.getMessage());
+    }
+  }
+
+  protected String readToken(StringBuffer chars) {
+    String word;
+    boolean space = false;
+    int i;
+    for (i = 0; i < chars.length(); i++) {
+      if (Character.isWhitespace(chars.charAt(i))) {
+        space = true;
+      } else {
+        break;
+      }
+    }
+    if (space) {
+      // chars.delete(0,i);
+      for (int countr = i; countr < chars.length(); countr++) {
+        chars.setCharAt(countr - i, chars.charAt(countr));
+      }
+      chars.setLength(chars.length() - i);
+      if (token.length() > 0) {
+        word = token.toString();
+        token.setLength(0);
+        return word;
+      }
+    }
+    space = false;
+    for (i = 0; i < chars.length(); i++) {
+      if (Character.isWhitespace(chars.charAt(i))) {
+        space = true;
+        break;
+      }
+    }
+    token.append(chars.toString().substring(0, i));
+    // chars.delete(0,i);
+    for (int countr = i; countr < chars.length(); countr++) {
+      chars.setCharAt(countr - i, chars.charAt(countr));
+    }
+    chars.setLength(chars.length() - i);
+    if (space) {
+      word = token.toString();
+      token.setLength(0);
+      return word;
+    }
+    token.append(chars);
+    return null;
+  }
+
+  protected static String getPattern(String word) {
+    StringBuffer pat = new StringBuffer();
+    int len = word.length();
+    for (int i = 0; i < len; i++) {
+      if (!Character.isDigit(word.charAt(i))) {
+        pat.append(word.charAt(i));
+      }
+    }
+    return pat.toString();
+  }
+
+  protected ArrayList normalizeException(ArrayList ex) {
+    ArrayList res = new ArrayList();
+    for (int i = 0; i < ex.size(); i++) {
+      Object item = ex.get(i);
+      if (item instanceof String) {
+        String str = (String) item;
+        StringBuffer buf = new StringBuffer();
+        for (int j = 0; j < str.length(); j++) {
+          char c = str.charAt(j);
+          if (c != hyphenChar) {
+            buf.append(c);
+          } else {
+            res.add(buf.toString());
+            buf.setLength(0);
+            char[] h = new char[1];
+            h[0] = hyphenChar;
+            // we use here hyphenChar which is not necessarily
+            // the one to be printed
+            res.add(new Hyphen(new String(h), null, null));
+          }
+        }
+        if (buf.length() > 0) {
+          res.add(buf.toString());
+        }
+      } else {
+        res.add(item);
+      }
+    }
+    return res;
+  }
+
+  protected String getExceptionWord(ArrayList ex) {
+    StringBuffer res = new StringBuffer();
+    for (int i = 0; i < ex.size(); i++) {
+      Object item = ex.get(i);
+      if (item instanceof String) {
+        res.append((String) item);
+      } else {
+        if (((Hyphen) item).noBreak != null) {
+          res.append(((Hyphen) item).noBreak);
+        }
+      }
+    }
+    return res.toString();
+  }
+
+  protected static String getInterletterValues(String pat) {
+    StringBuffer il = new StringBuffer();
+    String word = pat + "a"; // add dummy letter to serve as sentinel
+    int len = word.length();
+    for (int i = 0; i < len; i++) {
+      char c = word.charAt(i);
+      if (Character.isDigit(c)) {
+        il.append(c);
+        i++;
+      } else {
+        il.append('0');
+      }
+    }
+    return il.toString();
+  }
+
+  //
+  // EntityResolver methods
+  //
+  public InputSource resolveEntity(String publicId, String systemId)
+  throws SAXException, IOException {
+    return HyphenationDTDGenerator.generateDTD();
+  }
+
+  //
+  // ContentHandler methods
+  //
+
+  /**
+   * @see org.xml.sax.ContentHandler#startElement(java.lang.String,
+   *      java.lang.String, java.lang.String, org.xml.sax.Attributes)
+   */
+  public void startElement(String uri, String local, String raw,
+      Attributes attrs) {
+    if (local.equals("hyphen-char")) {
+      String h = attrs.getValue("value");
+      if (h != null && h.length() == 1) {
+        hyphenChar = h.charAt(0);
+      }
+    } else if (local.equals("classes")) {
+      currElement = ELEM_CLASSES;
+    } else if (local.equals("patterns")) {
+      currElement = ELEM_PATTERNS;
+    } else if (local.equals("exceptions")) {
+      currElement = ELEM_EXCEPTIONS;
+      exception = new ArrayList();
+    } else if (local.equals("hyphen")) {
+      if (token.length() > 0) {
+        exception.add(token.toString());
+      }
+      exception.add(new Hyphen(attrs.getValue("pre"), attrs.getValue("no"),
+          attrs.getValue("post")));
+      currElement = ELEM_HYPHEN;
+    }
+    token.setLength(0);
+  }
+
+  /**
+   * @see org.xml.sax.ContentHandler#endElement(java.lang.String,
+   *      java.lang.String, java.lang.String)
+   */
+  public void endElement(String uri, String local, String raw) {
+
+    if (token.length() > 0) {
+      String word = token.toString();
+      switch (currElement) {
+        case ELEM_CLASSES:
+          consumer.addClass(word);
+          break;
+        case ELEM_EXCEPTIONS:
+          exception.add(word);
+          exception = normalizeException(exception);
+          consumer.addException(getExceptionWord(exception),
+              (ArrayList) exception.clone());
+          break;
+        case ELEM_PATTERNS:
+          consumer.addPattern(getPattern(word), getInterletterValues(word));
+          break;
+        case ELEM_HYPHEN:
+          // nothing to do
+          break;
+      }
+      if (currElement != ELEM_HYPHEN) {
+        token.setLength(0);
+      }
+    }
+    if (currElement == ELEM_HYPHEN) {
+      currElement = ELEM_EXCEPTIONS;
+    } else {
+      currElement = 0;
+    }
+
+  }
+
+  /**
+   * @see org.xml.sax.ContentHandler#characters(char[], int, int)
+   */
+  public void characters(char ch[], int start, int length) {
+    StringBuffer chars = new StringBuffer(length);
+    chars.append(ch, start, length);
+    String word = readToken(chars);
+    while (word != null) {
+      // System.out.println("\"" + word + "\"");
+      switch (currElement) {
+        case ELEM_CLASSES:
+          consumer.addClass(word);
+          break;
+        case ELEM_EXCEPTIONS:
+          exception.add(word);
+          exception = normalizeException(exception);
+          consumer.addException(getExceptionWord(exception),
+              (ArrayList) exception.clone());
+          exception.clear();
+          break;
+        case ELEM_PATTERNS:
+          consumer.addPattern(getPattern(word), getInterletterValues(word));
+          break;
+      }
+      word = readToken(chars);
+    }
+
+  }
+
+  //
+  // ErrorHandler methods
+  //
+
+  /**
+   * @see org.xml.sax.ErrorHandler#warning(org.xml.sax.SAXParseException)
+   */
+  public void warning(SAXParseException ex) {
+    errMsg = "[Warning] " + getLocationString(ex) + ": " + ex.getMessage();
+  }
+
+  /**
+   * @see org.xml.sax.ErrorHandler#error(org.xml.sax.SAXParseException)
+   */
+  public void error(SAXParseException ex) {
+    errMsg = "[Error] " + getLocationString(ex) + ": " + ex.getMessage();
+  }
+
+  /**
+   * @see org.xml.sax.ErrorHandler#fatalError(org.xml.sax.SAXParseException)
+   */
+  public void fatalError(SAXParseException ex) throws SAXException {
+    errMsg = "[Fatal Error] " + getLocationString(ex) + ": " + ex.getMessage();
+    throw ex;
+  }
+
+  /**
+   * Returns a string of the location.
+   */
+  private String getLocationString(SAXParseException ex) {
+    StringBuffer str = new StringBuffer();
+
+    String systemId = ex.getSystemId();
+    if (systemId != null) {
+      int index = systemId.lastIndexOf('/');
+      if (index != -1) {
+        systemId = systemId.substring(index + 1);
+      }
+      str.append(systemId);
+    }
+    str.append(':');
+    str.append(ex.getLineNumber());
+    str.append(':');
+    str.append(ex.getColumnNumber());
+
+    return str.toString();
+
+  } // getLocationString(SAXParseException):String
+
+  // PatternConsumer implementation for testing purposes
+  public void addClass(String c) {
+    System.out.println("class: " + c);
+  }
+
+  public void addException(String w, ArrayList e) {
+    System.out.println("exception: " + w + " : " + e.toString());
+  }
+
+  public void addPattern(String p, String v) {
+    System.out.println("pattern: " + p + " : " + v);
+  }
+
+  public static void main(String[] args) throws Exception {
+    if (args.length > 0) {
+      PatternParser pp = new PatternParser();
+      pp.setConsumer(pp);
+      pp.parse(args[0]);
+    }
+  }
+}
+
+class HyphenationDTDGenerator {
+  public static final String DTD_STRING=
+    "<?xml version=\"1.0\" encoding=\"US-ASCII\"?>\n"+
+    "<!--\n"+
+    "  Copyright 1999-2004 The Apache Software Foundation\n"+
+    "\n"+
+    "  Licensed under the Apache License, Version 2.0 (the \"License\");\n"+
+    "  you may not use this file except in compliance with the License.\n"+
+    "  You may obtain a copy of the License at\n"+
+    "\n"+
+    "       http://www.apache.org/licenses/LICENSE-2.0\n"+
+    "\n"+
+    "  Unless required by applicable law or agreed to in writing, software\n"+
+    "  distributed under the License is distributed on an \"AS IS\" BASIS,\n"+
+    "  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n"+
+    "  See the License for the specific language governing permissions and\n"+
+    "  limitations under the License.\n"+
+    "-->\n"+
+    "<!-- $Id: hyphenation.dtd,v 1.3 2004/02/27 18:34:59 jeremias Exp $ -->\n"+
+    "\n"+
+    "<!ELEMENT hyphenation-info (hyphen-char?, hyphen-min?,\n"+
+    "                           classes, exceptions?, patterns)>\n"+
+    "\n"+
+    "<!-- Hyphen character to be used in the exception list as shortcut for\n"+
+    "     <hyphen pre-break=\"-\"/>. Defaults to '-'\n"+
+    "-->\n"+
+    "<!ELEMENT hyphen-char EMPTY>\n"+
+    "<!ATTLIST hyphen-char value CDATA #REQUIRED>\n"+
+    "\n"+
+    "<!-- Default minimun length in characters of hyphenated word fragments\n"+
+    "     before and after the line break. For some languages this is not\n"+
+    "     only for aesthetic purposes, wrong hyphens may be generated if this\n"+
+    "     is not accounted for.\n"+
+    "-->\n"+
+    "<!ELEMENT hyphen-min EMPTY>\n"+
+    "<!ATTLIST hyphen-min before CDATA #REQUIRED>\n"+
+    "<!ATTLIST hyphen-min after CDATA #REQUIRED>\n"+
+    "\n"+
+    "<!-- Character equivalent classes: space separated list of character groups, all\n"+
+    "     characters in a group are to be treated equivalent as far as\n"+
+    "     the hyphenation algorithm is concerned. The first character in a group\n"+
+    "     is the group's equivalent character. Patterns should only contain\n"+
+    "     first characters. It also defines word characters, i.e. a word that\n"+
+    "     contains characters not present in any of the classes is not hyphenated.\n"+
+    "-->\n"+
+    "<!ELEMENT classes (#PCDATA)>\n"+
+    "\n"+
+    "<!-- Hyphenation exceptions: space separated list of hyphenated words.\n"+
+    "     A hyphen is indicated by the hyphen tag, but you can use the\n"+
+    "     hyphen-char defined previously as shortcut. This is in cases\n"+
+    "     when the algorithm procedure finds wrong hyphens or you want\n"+
+    "     to provide your own hyphenation for some words.\n"+
+    "-->\n"+
+    "<!ELEMENT exceptions (#PCDATA|hyphen)* >\n"+
+    "\n"+
+    "<!-- The hyphenation patterns, space separated. A pattern is made of 'equivalent'\n"+
+    "     characters as described before, between any two word characters a digit\n"+
+    "     in the range 0 to 9 may be specified. The absence of a digit is equivalent\n"+
+    "     to zero. The '.' character is reserved to indicate begining or ending\n"+
+    "     of words. -->\n"+
+    "<!ELEMENT patterns (#PCDATA)>\n"+
+    "\n"+
+    "<!-- A \"full hyphen\" equivalent to TeX's \\discretionary\n"+
+    "     with pre-break, post-break and no-break attributes.\n"+
+    "     To be used in the exceptions list, the hyphen character is not\n"+
+    "     automatically added -->\n"+
+    "<!ELEMENT hyphen EMPTY>\n"+
+    "<!ATTLIST hyphen pre CDATA #IMPLIED>\n"+
+    "<!ATTLIST hyphen no CDATA #IMPLIED>\n"+
+    "<!ATTLIST hyphen post CDATA #IMPLIED>\n";
+  
+ public static InputSource generateDTD() {
+    return new InputSource(new StringReader(DTD_STRING));
+  }
+}

Propchange: lucene/java/trunk/contrib/analyzers/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java
------------------------------------------------------------------------------
    svn:eol-style = native