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 2010/11/17 13:26:16 UTC

svn commit: r1035996 - in /lucene/dev/trunk: modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/ modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/ solr/ solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/ so...

Author: rmuir
Date: Wed Nov 17 12:26:15 2010
New Revision: 1035996

URL: http://svn.apache.org/viewvc?rev=1035996&view=rev
Log:
SOLR-2237: Added StempelPolishStemFilterFactory to contrib/analysis-extras

Added:
    lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java   (with props)
    lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java   (with props)
Modified:
    lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
    lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/StempelStemmer.java
    lucene/dev/trunk/solr/CHANGES.txt

Modified: lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java?rev=1035996&r1=1035995&r2=1035996&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java (original)
+++ lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/pl/PolishAnalyzer.java Wed Nov 17 12:26:15 2010
@@ -17,8 +17,6 @@ package org.apache.lucene.analysis.pl;
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
@@ -50,6 +48,9 @@ public final class PolishAnalyzer extend
   /** File containing default Polish stopwords. */
   public final static String DEFAULT_STOPWORD_FILE = "stopwords.txt";
   
+  /** File containing default Polish stemmer table. */
+  public final static String DEFAULT_STEMMER_FILE = "stemmer_20000.tbl";
+  
   /**
    * Returns an unmodifiable instance of the default stop words set.
    * @return default stop words set.
@@ -76,16 +77,8 @@ public final class PolishAnalyzer extend
         throw new RuntimeException("Unable to load default stopword set", ex);
       }
       
-      InputStream stream = PolishAnalyzer.class.getResourceAsStream("stemmer_20000.tbl");
       try {
-        DataInputStream in = new DataInputStream(new BufferedInputStream(stream));
-        String method = in.readUTF().toUpperCase();
-        if (method.indexOf('M') < 0) {
-          DEFAULT_TABLE = new org.egothor.stemmer.Trie(in);
-        } else {
-          DEFAULT_TABLE = new org.egothor.stemmer.MultiTrie2(in);
-        }
-        in.close();
+        DEFAULT_TABLE = StempelStemmer.load(PolishAnalyzer.class.getResourceAsStream(DEFAULT_STEMMER_FILE));
       } catch (IOException ex) {
         // default set should always be present as it is part of the
         // distribution (JAR)

Modified: lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/StempelStemmer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/StempelStemmer.java?rev=1035996&r1=1035995&r2=1035996&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/StempelStemmer.java (original)
+++ lucene/dev/trunk/modules/analysis/stempel/src/java/org/apache/lucene/analysis/stempel/StempelStemmer.java Wed Nov 17 12:26:15 2010
@@ -20,6 +20,7 @@ import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.Locale;
 
 import org.egothor.stemmer.Diff;
 import org.egothor.stemmer.Trie;
@@ -45,17 +46,7 @@ public class StempelStemmer {
    * @param stemmerTable stemmer table.
    */
   public StempelStemmer(InputStream stemmerTable) throws IOException {
-    if (stemmerTable == null) return;
-    
-    DataInputStream in = new DataInputStream(new BufferedInputStream(
-        stemmerTable));
-    String method = in.readUTF().toUpperCase();
-    if (method.indexOf('M') < 0) {
-      stemmer = new org.egothor.stemmer.Trie(in);
-    } else {
-      stemmer = new org.egothor.stemmer.MultiTrie2(in);
-    }
-    in.close();
+    this(load(stemmerTable));
   }
 
   /**
@@ -66,6 +57,24 @@ public class StempelStemmer {
   public StempelStemmer(Trie stemmer) {
     this.stemmer = stemmer;
   }
+  
+  /**
+   * Load a stemmer table from an inputstream.
+   */
+  public static Trie load(InputStream stemmerTable) throws IOException {
+    DataInputStream in = null;
+    try {
+      in = new DataInputStream(new BufferedInputStream(stemmerTable));
+      String method = in.readUTF().toUpperCase(Locale.ENGLISH);
+      if (method.indexOf('M') < 0) {
+        return new org.egothor.stemmer.Trie(in);
+      } else {
+        return new org.egothor.stemmer.MultiTrie2(in);
+      }
+    } finally {
+      in.close();
+    }
+  }
 
   /**
    * Stem a word. 

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1035996&r1=1035995&r2=1035996&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Wed Nov 17 12:26:15 2010
@@ -305,6 +305,8 @@ New Features
 * SOLR-2211: Added UAX29TokenizerFactory, which implements UAX#29, a unicode algorithm 
   with good results for most languages.  (Tom Burton-West via rmuir)
 
+* SOLR-2237: Added StempelPolishStemFilterFactory to contrib/analysis-extras (rmuir)
+
 Optimizations
 ----------------------
 

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java?rev=1035996&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java Wed Nov 17 12:26:15 2010
@@ -0,0 +1,49 @@
+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 org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.stempel.StempelFilter;
+import org.apache.lucene.analysis.stempel.StempelStemmer;
+import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.util.plugin.ResourceLoaderAware;
+import org.egothor.stemmer.Trie;
+
+/**
+ * Factory for {@link StempelFilter} using a Polish stemming table.
+ */
+public class StempelPolishStemFilterFactory extends BaseTokenFilterFactory implements ResourceLoaderAware {
+  private Trie stemmer = null;
+  private static final String STEMTABLE = "org/apache/lucene/analysis/pl/stemmer_20000.tbl";
+  
+  public TokenStream create(TokenStream input) {
+    return new StempelFilter(input, new StempelStemmer(stemmer));
+  }
+
+  public void inform(ResourceLoader loader) {
+    try {
+      stemmer = StempelStemmer.load(loader.openResource(STEMTABLE));
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load stem table: " + STEMTABLE);
+    }
+  }
+}

Propchange: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/analysis/StempelPolishStemFilterFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java?rev=1035996&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java (added)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java Wed Nov 17 12:26:15 2010
@@ -0,0 +1,38 @@
+package org.apache.solr.analysis;
+
+import java.io.StringReader;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+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.
+ */
+
+/**
+ * Tests for {@link StempelPolishStemFilterFactory}
+ */
+public class TestStempelPolishStemFilterFactory extends BaseTokenTestCase {
+  public void testBasics() throws Exception {
+    StringReader document = new StringReader("studenta studenci");
+    StempelPolishStemFilterFactory factory = new StempelPolishStemFilterFactory();
+    factory.inform(new SolrResourceLoader(null, null));
+    TokenStream ts = factory.create(new WhitespaceTokenizer(DEFAULT_VERSION, document));
+    assertTokenStreamContents(ts,
+        new String[] { "student", "student" });
+  }
+}

Propchange: lucene/dev/trunk/solr/contrib/analysis-extras/src/test/org/apache/solr/analysis/TestStempelPolishStemFilterFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native