You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/03/16 19:11:10 UTC

svn commit: r1578133 [11/11] - in /lucene/dev/branches/lucene5376_2/lucene: ./ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/pattern/ analysis/common/src/java/org/apache/lucene/analy...

Modified: lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java?rev=1578133&r1=1578132&r2=1578133&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java Sun Mar 16 18:11:07 2014
@@ -195,7 +195,6 @@ public class AnalyzingInfixSuggester ext
 
   @Override
   public void build(InputIterator iter) throws IOException {
-
     if (searcherMgr != null) {
       searcherMgr.close();
       searcherMgr = null;
@@ -316,6 +315,9 @@ public class AnalyzingInfixSuggester ext
    *  you must call {@link #refresh} in the end in order to
    *  see the suggestions in {@link #lookup} */
   public void add(BytesRef text, long weight, BytesRef payload) throws IOException {
+    if (searcherMgr == null) {
+      throw new IllegalStateException("suggester was not built");
+    }
     String textString = text.utf8ToString();
     Document doc = new Document();
     FieldType ft = getTextFieldType();
@@ -338,6 +340,9 @@ public class AnalyzingInfixSuggester ext
    *  new suggestions, you must call {@link #refresh} in the
    *  end in order to see the suggestions in {@link #lookup} */
   public void update(BytesRef text, long weight, BytesRef payload) throws IOException {
+    if (searcherMgr == null) {
+      throw new IllegalStateException("suggester was not built");
+    }
     String textString = text.utf8ToString();
     Document doc = new Document();
     FieldType ft = getTextFieldType();
@@ -356,9 +361,20 @@ public class AnalyzingInfixSuggester ext
    *  up" many additions/updates, and then call refresh
    *  once in the end. */
   public void refresh() throws IOException {
+    if (searcherMgr == null) {
+      throw new IllegalStateException("suggester was not built");
+    }
     searcherMgr.maybeRefreshBlocking();
   }
 
+  /** Commits changes to the underlying index. */
+  public void commit() throws IOException {
+    if (searcherMgr == null) {
+      throw new IllegalStateException("suggester was not built");
+    }
+    writer.commit();
+  }
+
   private void initSorter() {
     sorter = new Sorter() {
 

Added: lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilterFactory.java?rev=1578133&view=auto
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilterFactory.java (added)
+++ lucene/dev/branches/lucene5376_2/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/SuggestStopFilterFactory.java Sun Mar 16 18:11:07 2014
@@ -0,0 +1,115 @@
+package org.apache.lucene.search.suggest.analyzing;
+
+/*
+ * 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.Map;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.StopAnalyzer;
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.lucene.analysis.util.TokenFilterFactory;
+import org.apache.lucene.analysis.util.WordlistLoader; // jdocs
+
+/**
+ * Factory for {@link SuggestStopFilter}.
+ *
+ * <p>
+ * All attributes are optional:
+ * </p>
+ * <ul>
+ *  <li><code>ignoreCase</code> defaults to <code>false</code></li>
+ *  <li><code>words</code> should be the name of a stopwords file to parse, if not 
+ *      specified the factory will use {@link StopAnalyzer#ENGLISH_STOP_WORDS_SET}
+ *  </li>
+ *  <li><code>format</code> defines how the <code>words</code> file will be parsed, 
+ *      and defaults to <code>wordset</code>.  If <code>words</code> is not specified, 
+ *      then <code>format</code> must not be specified.
+ *  </li>
+ * </ul>
+ * <p>
+ * The valid values for the <code>format</code> option are:
+ * </p>
+ * <ul>
+ *  <li><code>wordset</code> - This is the default format, which supports one word per 
+ *      line (including any intra-word whitespace) and allows whole line comments 
+ *      begining with the "#" character.  Blank lines are ignored.  See 
+ *      {@link WordlistLoader#getLines WordlistLoader.getLines} for details.
+ *  </li>
+ *  <li><code>snowball</code> - This format allows for multiple words specified on each 
+ *      line, and trailing comments may be specified using the vertical line ("&#124;"). 
+ *      Blank lines are ignored.  See 
+ *      {@link WordlistLoader#getSnowballWordSet WordlistLoader.getSnowballWordSet} 
+ *      for details.
+ *  </li>
+ * </ul>
+ */
+public class SuggestStopFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
+  public static final String FORMAT_WORDSET = "wordset";
+  public static final String FORMAT_SNOWBALL = "snowball";
+  
+  private CharArraySet stopWords;
+  private final String stopWordFiles;
+  private final String format;
+  private final boolean ignoreCase;
+  
+  /** Creates a new StopFilterFactory */
+  public SuggestStopFilterFactory(Map<String,String> args) {
+    super(args);
+    assureMatchVersion();
+    stopWordFiles = get(args, "words");
+    format = get(args, "format", (null == stopWordFiles ? null : FORMAT_WORDSET));
+    ignoreCase = getBoolean(args, "ignoreCase", false);
+    if (!args.isEmpty()) {
+      throw new IllegalArgumentException("Unknown parameters: " + args);
+    }
+  }
+
+  @Override
+  public void inform(ResourceLoader loader) throws IOException {
+    if (stopWordFiles != null) {
+      if (FORMAT_WORDSET.equalsIgnoreCase(format)) {
+        stopWords = getWordSet(loader, stopWordFiles, ignoreCase);
+      } else if (FORMAT_SNOWBALL.equalsIgnoreCase(format)) {
+        stopWords = getSnowballWordSet(loader, stopWordFiles, ignoreCase);
+      } else {
+        throw new IllegalArgumentException("Unknown 'format' specified for 'words' file: " + format);
+      }
+    } else {
+      if (null != format) {
+        throw new IllegalArgumentException("'format' can not be specified w/o an explicit 'words' file: " + format);
+      }
+      stopWords = new CharArraySet(luceneMatchVersion, StopAnalyzer.ENGLISH_STOP_WORDS_SET, ignoreCase);
+    }
+  }
+
+  public boolean isIgnoreCase() {
+    return ignoreCase;
+  }
+
+  public CharArraySet getStopWords() {
+    return stopWords;
+  }
+
+  @Override
+  public TokenStream create(TokenStream input) {
+    return new SuggestStopFilter(input, stopWords);
+  }
+}