You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2010/08/23 15:44:01 UTC

svn commit: r988120 - in /lucene/dev/trunk/solr: ./ src/java/org/apache/solr/handler/component/ src/java/org/apache/solr/spelling/ src/java/org/apache/solr/spelling/suggest/ src/java/org/apache/solr/spelling/suggest/jaspell/ src/java/org/apache/solr/sp...

Author: ab
Date: Mon Aug 23 13:44:00 2010
New Revision: 988120

URL: http://svn.apache.org/viewvc?rev=988120&view=rev
Log:
SOLR-1316 Create autosuggest component.

Added:
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java   (with props)
    lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java   (with props)
    lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/
    lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/SolrSpellChecker.java
    lucene/dev/trunk/solr/src/java/org/apache/solr/util/HighFrequencyDictionary.java
    lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DummyCustomParamSpellChecker.java
    lucene/dev/trunk/solr/src/test/test-files/solr/conf/schema-spellchecker.xml
    lucene/dev/trunk/solr/src/test/test-files/solr/conf/solrconfig-spellchecker.xml

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Aug 23 13:44:00 2010
@@ -237,6 +237,9 @@ New Features
    Koji Sekiguchi, Iván de Prado, Ryan McKinley, Marc Sturlese, Peter Karich,
    Bojan Smid, Charles Hornberger, Dieter Grad, Dmitry Lihachev, Doug Steigerwald,
    Karsten Sperling, Michael Gundlach, Oleg Gnatovskiy, Thomas Traeger, yonik)
+
+* SOLR-1316: Create autosuggest component.
+  (Ankul Garg, Jason Rutherglen, Shalin Shekhar Mangar, Grant Ingersoll, Robert Muir, ab)
    
 
 Optimizations

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Mon Aug 23 13:44:00 2010
@@ -111,7 +111,7 @@ public class SpellCheckComponent extends
       spellChecker.build(rb.req.getCore(), rb.req.getSearcher());
       rb.rsp.add("command", "build");
     } else if (params.getBool(SPELLCHECK_RELOAD, false)) {
-      spellChecker.reload();
+      spellChecker.reload(rb.req.getCore(), rb.req.getSearcher());
       rb.rsp.add("command", "reload");
     }
   }
@@ -555,7 +555,7 @@ public class SpellCheckComponent extends
         try {
           LOG.info("Loading spell index for spellchecker: "
                   + checker.getDictionaryName());
-          checker.reload();
+          checker.reload(core, newSearcher);
         } catch (IOException e) {
           log.error( "Exception in reloading spell check index for spellchecker: " + checker.getDictionaryName(), e);
         }

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java Mon Aug 23 13:44:00 2010
@@ -44,6 +44,7 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.search.SolrIndexSearcher;
 
 
 /**
@@ -195,7 +196,7 @@ public abstract class AbstractLuceneSpel
     return reader;
   }
 
-  public void reload() throws IOException {
+  public void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException {
     spellChecker.setSpellIndex(index);
 
   }

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java Mon Aug 23 13:44:00 2010
@@ -102,8 +102,8 @@ public class IndexBasedSpellChecker exte
   }
 
   @Override
-  public void reload() throws IOException {
-    super.reload();
+  public void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException {
+    super.reload(core, searcher);
     //reload the source
     initSourceReader();
   }

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/SolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/SolrSpellChecker.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/SolrSpellChecker.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/SolrSpellChecker.java Mon Aug 23 13:44:00 2010
@@ -63,7 +63,7 @@ public abstract class SolrSpellChecker {
    *
    * @throws java.io.IOException
    */
-  public abstract void reload() throws IOException;
+  public abstract void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException;
 
   /**
    * (re)Builds the spelling index.  May be a NOOP if the implementation doesn't require building, or can't be rebuilt.

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,70 @@
+package org.apache.solr.spelling.suggest;
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.util.TermFreqIterator;
+
+/**
+ * This wrapper buffers incoming elements.
+ */
+public class BufferingTermFreqIteratorWrapper implements TermFreqIterator {
+
+  /** Entry in the buffer. */
+  public static final class Entry implements Comparable<Entry> {
+    String word;
+    float freq;
+    
+    public Entry(String word, float freq) {
+      this.word = word;
+      this.freq = freq;
+    }
+    
+    @Override
+    public int compareTo(Entry o) {
+      return word.compareTo(o.word);
+    }    
+  }
+
+  protected ArrayList<Entry> entries = new ArrayList<Entry>();
+  
+  protected int curPos;
+  protected Entry curEntry;
+  
+  public BufferingTermFreqIteratorWrapper(TermFreqIterator source) {
+    // read all source data into buffer
+    while (source.hasNext()) {
+      String w = source.next();
+      Entry e = new Entry(w, source.freq());
+      entries.add(e);
+    }
+    curPos = 0;
+  }
+
+  @Override
+  public float freq() {
+    return curEntry.freq;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return curPos < entries.size();
+  }
+
+  @Override
+  public String next() {
+    curEntry = entries.get(curPos);
+    curPos++;
+    return curEntry.word;
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException("remove is not supported");
+  }
+  
+  public List<Entry> entries() {
+    return entries;
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/BufferingTermFreqIteratorWrapper.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,95 @@
+package org.apache.solr.spelling.suggest;
+
+/**
+ * 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.*;
+
+import org.apache.lucene.search.spell.Dictionary;
+import org.apache.solr.util.TermFreqIterator;
+
+
+/**
+ * Dictionary represented by a text file.
+ * 
+ * <p/>Format allowed: 1 string per line, optionally with a tab-separated integer value:<br/>
+ * word1 TAB 100<br/>
+ * word2 word3 TAB 101<br/>
+ * word4 word5 TAB 102<br/>
+ */
+public class FileDictionary implements Dictionary {
+
+  private BufferedReader in;
+  private String line;
+  private boolean hasNextCalled;
+
+  public FileDictionary(InputStream dictFile) {
+    in = new BufferedReader(new InputStreamReader(dictFile));
+  }
+
+  /**
+   * Creates a dictionary based on a reader.
+   */
+  public FileDictionary(Reader reader) {
+    in = new BufferedReader(reader);
+  }
+
+  public TermFreqIterator getWordsIterator() {
+    return new fileIterator();
+  }
+
+  final class fileIterator implements TermFreqIterator {
+    private float curFreq;
+    
+    public String next() {
+      if (!hasNextCalled) {
+        hasNext();
+      }
+      hasNextCalled = false;
+      return line;
+    }
+    
+    public float freq() {
+      return curFreq;
+    }
+
+    public boolean hasNext() {
+      hasNextCalled = true;
+      try {
+        line = in.readLine();
+        if (line != null) {
+          String[] fields = line.split("\t");
+          if (fields.length > 1) {
+            curFreq = Float.parseFloat(fields[1]);
+            line = fields[0];
+          } else {
+            curFreq = 1;
+          }
+        }
+      } catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+      return (line != null) ? true : false;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/FileDictionary.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,117 @@
+package org.apache.solr.spelling.suggest;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.search.spell.Dictionary;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.TermFreqIterator;
+
+public abstract class Lookup {
+  
+  /**
+   * Result of a lookup.
+   */
+  public static final class LookupResult {
+    String key;
+    float value;
+    
+    public LookupResult(String key, float value) {
+      this.key = key;
+      this.value = value;
+    }
+    
+    public String toString() {
+      return key + "/" + value;
+    }
+  }
+  
+  public static final class LookupPriorityQueue extends PriorityQueue<LookupResult> {
+    
+    public LookupPriorityQueue(int size) {
+      initialize(size);
+    }
+
+    @Override
+    protected boolean lessThan(LookupResult a, LookupResult b) {
+      return a.value < b.value;
+    }
+    
+    public LookupResult[] getResults() {
+      int size = size();
+      LookupResult[] res = new LookupResult[size];
+      for (int i = size - 1; i >= 0; i--) {
+        res[i] = pop();
+      }
+      return res;
+    }
+  }
+  
+  /** Initialize the lookup. */
+  public abstract void init(NamedList config, SolrCore core);
+  
+  /** Build lookup from a dictionary. Some implementations may require sorted
+   * or unsorted keys from the dictionary's iterator - use
+   * {@link SortedTermFreqIteratorWrapper} or
+   * {@link UnsortedTermFreqIteratorWrapper} in such case.
+   */
+  public void build(Dictionary dict) throws IOException {
+    Iterator<String> it = dict.getWordsIterator();
+    TermFreqIterator tfit;
+    if (it instanceof TermFreqIterator) {
+      tfit = (TermFreqIterator)it;
+    } else {
+      tfit = new TermFreqIterator.TermFreqIteratorWrapper(it);
+    }
+    build(tfit);
+  }
+  
+  protected abstract void build(TermFreqIterator tfit) throws IOException;
+  
+  /**
+   * Persist the constructed lookup data to a directory. Optional operation.
+   * @param storeDir directory where data can be stored.
+   * @return true if successful, false if unsuccessful or not supported.
+   * @throws IOException when fatal IO error occurs.
+   */
+  public abstract boolean store(File storeDir) throws IOException;
+
+  /**
+   * Discard current lookup data and load it from a previously saved copy.
+   * Optional operation.
+   * @param storeDir directory where lookup data was stored.
+   * @return true if completed successfully, false if unsuccessful or not supported.
+   * @throws IOException when fatal IO error occurs.
+   */
+  public abstract boolean load(File storeDir) throws IOException;
+  
+  /**
+   * Look up a key and return possible completion for this key.
+   * @param key lookup key. Depending on the implementation this may be
+   * a prefix, misspelling, or even infix.
+   * @param onlyMorePopular return only more popular results
+   * @param num maximum number of results to return
+   * @return a list of possible completions, with their relative weight (e.g. popularity)
+   */
+  public abstract List<LookupResult> lookup(String key, boolean onlyMorePopular, int num);
+
+  /**
+   * Modify the lookup data by recording additional data. Optional operation.
+   * @param key new lookup key
+   * @param value value to associate with this key
+   * @return true if new key is added, false if it already exists or operation
+   * is not supported.
+   */
+  public abstract boolean add(String key, Object value);
+  
+  /**
+   * Get value associated with a specific key.
+   * @param key lookup key
+   * @return associated value
+   */
+  public abstract Object get(String key);  
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,18 @@
+package org.apache.solr.spelling.suggest;
+
+import java.util.Collections;
+
+import org.apache.solr.util.SortedIterator;
+import org.apache.solr.util.TermFreqIterator;
+
+/**
+ * This wrapper buffers incoming elements and makes sure they are sorted in
+ * ascending lexicographic order.
+ */
+public class SortedTermFreqIteratorWrapper extends BufferingTermFreqIteratorWrapper implements SortedIterator {
+
+  public SortedTermFreqIteratorWrapper(TermFreqIterator source) {
+    super(source);
+    Collections.sort(entries);
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/SortedTermFreqIteratorWrapper.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,158 @@
+package org.apache.solr.spelling.suggest;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.spell.Dictionary;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.spelling.SolrSpellChecker;
+import org.apache.solr.spelling.SpellingOptions;
+import org.apache.solr.spelling.SpellingResult;
+import org.apache.solr.spelling.suggest.Lookup.LookupResult;
+import org.apache.solr.spelling.suggest.jaspell.JaspellLookup;
+import org.apache.solr.util.HighFrequencyDictionary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Suggester extends SolrSpellChecker {
+  private static final Logger LOG = LoggerFactory.getLogger(Suggester.class);
+  
+  /** Location of the source data - either a path to a file, or null for the
+   * current IndexReader.
+   */
+  public static final String LOCATION = "sourceLocation";
+  /** Field to use as the source of terms if using IndexReader. */
+  public static final String FIELD = "field";
+  /** Fully-qualified class of the {@link Lookup} implementation. */
+  public static final String LOOKUP_IMPL = "lookupImpl";
+  /**
+   * Minimum frequency of terms to consider when building the dictionary.
+   */
+  public static final String THRESHOLD_TOKEN_FREQUENCY = "threshold";
+  /**
+   * Name of the location where to persist the dictionary. If this location
+   * is relative then the data will be stored under the core's dataDir. If this
+   * is null the storing will be disabled.
+   */
+  public static final String STORE_DIR = "storeDir";
+  
+  protected String sourceLocation;
+  protected File storeDir;
+  protected String field;
+  protected float threshold;
+  protected Dictionary dictionary;
+  protected IndexReader reader;
+  protected Lookup lookup;
+  protected String lookupImpl;
+  protected SolrCore core;
+  
+  @Override
+  public String init(NamedList config, SolrCore core) {
+    LOG.info("init: " + config);
+    String name = super.init(config, core);
+    threshold = config.get(THRESHOLD_TOKEN_FREQUENCY) == null ? 0.0f
+            : (Float) config.get(THRESHOLD_TOKEN_FREQUENCY);
+    sourceLocation = (String) config.get(LOCATION);
+    field = (String)config.get(FIELD);
+    lookupImpl = (String)config.get(LOOKUP_IMPL);
+    if (lookupImpl == null) {
+      lookupImpl = JaspellLookup.class.getName();
+    }
+    String store = (String)config.get(STORE_DIR);
+    if (store != null) {
+      storeDir = new File(store);
+      if (!storeDir.isAbsolute()) {
+        storeDir = new File(core.getDataDir() + File.separator + storeDir);
+      }
+      if (!storeDir.exists()) {
+        storeDir.mkdirs();
+      }
+    }
+    return name;
+  }
+  
+  @Override
+  public void build(SolrCore core, SolrIndexSearcher searcher) {
+    LOG.info("build()");
+    if (sourceLocation == null) {
+      reader = searcher.getReader();
+      dictionary = new HighFrequencyDictionary(reader, field, threshold);
+    } else {
+      try {
+        dictionary = new FileDictionary(new InputStreamReader(
+                core.getResourceLoader().openResource(sourceLocation), "UTF-8"));
+      } catch (UnsupportedEncodingException e) {
+        e.printStackTrace();
+      }
+    }
+    lookup = (Lookup) core.getResourceLoader().newInstance(lookupImpl);
+    try {
+      lookup.build(dictionary);
+      if (storeDir != null) {
+        lookup.store(storeDir);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException {
+    LOG.info("reload()");
+    if (dictionary == null && storeDir != null) {
+      // this may be a firstSearcher event, try loading it
+      if (lookup.load(storeDir)) {
+        return;  // loaded ok
+      }
+    }
+    // dictionary based on the current index may need refreshing
+    if (dictionary instanceof HighFrequencyDictionary) {
+      reader = reader.reopen();
+      dictionary = new HighFrequencyDictionary(reader, field, threshold);
+      try {
+        lookup.build(dictionary);
+        if (storeDir != null) {
+          lookup.store(storeDir);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  public void add(String query, int numHits) {
+    LOG.info("add " + query + ", " + numHits);
+    lookup.add(query, new Integer(numHits));
+  }
+  
+  static SpellingResult EMPTY_RESULT = new SpellingResult();
+
+  @Override
+  public SpellingResult getSuggestions(SpellingOptions options) throws IOException {
+    LOG.debug("getSuggestions: " + options.tokens);
+    if (lookup == null) {
+      LOG.info("Lookup is null - invoke spellchecker.build first");
+      return EMPTY_RESULT;
+    }
+    SpellingResult res = new SpellingResult();
+    for (Token t : options.tokens) {
+      String term = new String(t.buffer(), 0, t.length());
+      List<LookupResult> suggestions = lookup.lookup(term,
+          options.onlyMorePopular, options.count);
+      if (suggestions == null) {
+        continue;
+      }
+      for (LookupResult lr : suggestions) {
+        res.add(t, lr.key, ((Number)lr.value).intValue());
+      }
+    }
+    return res;
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,17 @@
+package org.apache.solr.spelling.suggest;
+
+import java.util.Collections;
+
+import org.apache.solr.util.TermFreqIterator;
+
+/**
+ * This wrapper buffers the incoming elements and makes sure they are in
+ * random order.
+ */
+public class UnsortedTermFreqIteratorWrapper extends BufferingTermFreqIteratorWrapper {
+
+  public UnsortedTermFreqIteratorWrapper(TermFreqIterator source) {
+    super(source);
+    Collections.shuffle(entries);
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/UnsortedTermFreqIteratorWrapper.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,102 @@
+package org.apache.solr.spelling.suggest.jaspell;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.spelling.suggest.Lookup;
+import org.apache.solr.spelling.suggest.UnsortedTermFreqIteratorWrapper;
+import org.apache.solr.util.SortedIterator;
+import org.apache.solr.util.TermFreqIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JaspellLookup extends Lookup {
+  private static final Logger LOG = LoggerFactory.getLogger(JaspellLookup.class);
+  JaspellTernarySearchTrie trie;
+  private boolean usePrefix = true;
+  private int editDistance = 2;
+
+  @Override
+  public void init(NamedList config, SolrCore core) {
+    LOG.info("init: " + config);
+  }
+
+  @Override
+  public void build(TermFreqIterator tfit) throws IOException {
+    if (tfit instanceof SortedIterator) {
+      // make sure it's unsorted
+      tfit = new UnsortedTermFreqIteratorWrapper(tfit);
+    }
+    trie = new JaspellTernarySearchTrie();
+    trie.setMatchAlmostDiff(editDistance);
+    while (tfit.hasNext()) {
+      String key = tfit.next();
+      float freq = tfit.freq();
+      if (key.length() == 0) {
+        continue;
+      }
+      trie.put(key, new Float(freq));
+    }
+  }
+
+  @Override
+  public boolean add(String key, Object value) {
+    trie.put(key, value);
+    // XXX
+    return false;
+  }
+
+  @Override
+  public Object get(String key) {
+    return trie.get(key);
+  }
+
+  @Override
+  public List<LookupResult> lookup(String key, boolean onlyMorePopular, int num) {
+    List<LookupResult> res = new ArrayList<LookupResult>();
+    List<String> list;
+    int count = onlyMorePopular ? num * 2 : num;
+    if (usePrefix) {
+      list = trie.matchPrefix(key, count);
+    } else {
+      list = trie.matchAlmost(key, count);
+    }
+    if (list == null || list.size() == 0) {
+      return res;
+      
+    }
+    int maxCnt = Math.min(num, list.size());
+    if (onlyMorePopular) {
+      LookupPriorityQueue queue = new LookupPriorityQueue(num);
+      for (String s : list) {
+        float freq = (Float)trie.get(s);
+        queue.insertWithOverflow(new LookupResult(s, freq));
+      }
+      for (LookupResult lr : queue.getResults()) {
+        res.add(lr);
+      }
+    } else {
+      for (int i = 0; i < maxCnt; i++) {
+        String s = list.get(i);
+        float freq = (Float)trie.get(s);
+        res.add(new LookupResult(s, freq));
+      }      
+    }
+    return res;
+  }
+
+  @Override
+  public boolean load(File storeDir) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean store(File storeDir) throws IOException {
+    return false;    
+  }
+
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,827 @@
+package org.apache.solr.spelling.suggest.jaspell;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.List;
+import java.util.Vector;
+import java.util.zip.GZIPInputStream;
+
+/**
+ * Implementation of a Ternary Search Trie, a data structure for storing
+ * <code>String</code> objects that combines the compact size of a binary search
+ * tree with the speed of a digital search trie, and is therefore ideal for
+ * practical use in sorting and searching data.</p>
+ * <p>
+ * 
+ * This data structure is faster than hashing for many typical search problems,
+ * and supports a broader range of useful problems and operations. Ternary
+ * searches are faster than hashing and more powerful, too.
+ * </p>
+ * <p>
+ * 
+ * The theory of ternary search trees was described at a symposium in 1997 (see
+ * "Fast Algorithms for Sorting and Searching Strings," by J.L. Bentley and R.
+ * Sedgewick, Proceedings of the 8th Annual ACM-SIAM Symposium on Discrete
+ * Algorithms, January 1997). Algorithms in C, Third Edition, by Robert
+ * Sedgewick (Addison-Wesley, 1998) provides yet another view of ternary search
+ * trees.
+ * 
+ * @author Bruno Martins
+ * 
+ */
+public class JaspellTernarySearchTrie {
+
+  /**
+   * An inner class of Ternary Search Trie that represents a node in the trie.
+   */
+  protected final class TSTNode {
+
+    /** Index values for accessing relatives array. */
+    protected final static int PARENT = 0, LOKID = 1, EQKID = 2, HIKID = 3;
+
+    /** The key to the node. */
+    protected Object data;
+
+    /** The relative nodes. */
+    protected TSTNode[] relatives = new TSTNode[4];
+
+    /** The char used in the split. */
+    protected char splitchar;
+
+    /**
+     * Constructor method.
+     * 
+     *@param splitchar
+     *          The char used in the split.
+     *@param parent
+     *          The parent node.
+     */
+    protected TSTNode(char splitchar, TSTNode parent) {
+      this.splitchar = splitchar;
+      relatives[PARENT] = parent;
+    }
+  }
+
+  /**
+   * Compares characters by alfabetical order.
+   * 
+   *@param cCompare2
+   *          The first char in the comparison.
+   *@param cRef
+   *          The second char in the comparison.
+   *@return A negative number, 0 or a positive number if the second char is
+   *         less, equal or greater.
+   */
+  private static int compareCharsAlphabetically(char cCompare2, char cRef) {
+    return Character.toLowerCase(cCompare2) - Character.toLowerCase(cRef);
+  }
+  
+  /* what follows is the original Jaspell code. 
+  private static int compareCharsAlphabetically(int cCompare2, int cRef) {
+    int cCompare = 0;
+    if (cCompare2 >= 65) {
+      if (cCompare2 < 89) {
+        cCompare = (2 * cCompare2) - 65;
+      } else if (cCompare2 < 97) {
+        cCompare = cCompare2 + 24;
+      } else if (cCompare2 < 121) {
+        cCompare = (2 * cCompare2) - 128;
+      } else cCompare = cCompare2;
+    } else cCompare = cCompare2;
+    if (cRef < 65) {
+      return cCompare - cRef;
+    }
+    if (cRef < 89) {
+      return cCompare - ((2 * cRef) - 65);
+    }
+    if (cRef < 97) {
+      return cCompare - (cRef + 24);
+    }
+    if (cRef < 121) {
+      return cCompare - ((2 * cRef) - 128);
+    }
+    return cCompare - cRef;
+  }
+  */
+
+  /**
+   * The default number of values returned by the <code>matchAlmost</code>
+   * method.
+   */
+  private int defaultNumReturnValues = -1;
+
+  /**
+   * the number of differences allowed in a call to the
+   * <code>matchAlmostKey</code> method.
+   */
+  private int matchAlmostDiff;
+
+  /** The base node in the trie. */
+  private TSTNode rootNode;
+
+  /**
+   * Constructs an empty Ternary Search Trie.
+   */
+  public JaspellTernarySearchTrie() {
+  }
+
+  /**
+   * Constructs a Ternary Search Trie and loads data from a <code>File</code>
+   * into the Trie. The file is a normal text document, where each line is of
+   * the form word TAB float.
+   * 
+   *@param file
+   *          The <code>File</code> with the data to load into the Trie.
+   *@exception IOException
+   *              A problem occured while reading the data.
+   */
+  public JaspellTernarySearchTrie(File file) throws IOException {
+    this(file, false);
+  }
+
+  /**
+   * Constructs a Ternary Search Trie and loads data from a <code>File</code>
+   * into the Trie. The file is a normal text document, where each line is of
+   * the form "word TAB float".
+   * 
+   *@param file
+   *          The <code>File</code> with the data to load into the Trie.
+   *@param compression
+   *          If true, the file is compressed with the GZIP algorithm, and if
+   *          false, the file is a normal text document.
+   *@exception IOException
+   *              A problem occured while reading the data.
+   */
+  public JaspellTernarySearchTrie(File file, boolean compression)
+          throws IOException {
+    this();
+    BufferedReader in;
+    if (compression)
+      in = new BufferedReader(new InputStreamReader(new GZIPInputStream(
+              new FileInputStream(file))));
+    else in = new BufferedReader(new InputStreamReader((new FileInputStream(
+            file))));
+    String word;
+    int pos;
+    Float occur, one = new Float(1);
+    int numWords = 0;
+    while ((word = in.readLine()) != null) {
+      numWords++;
+      pos = word.indexOf("\t");
+      occur = one;
+      if (pos != -1) {
+        occur = Float.parseFloat(word.substring(pos + 1).trim());
+        word = word.substring(0, pos);
+      }
+      String key = word.toLowerCase();
+      if (rootNode == null) {
+        rootNode = new TSTNode(key.charAt(0), null);
+      }
+      TSTNode node = null;
+      if (key.length() > 0 && rootNode != null) {
+        TSTNode currentNode = rootNode;
+        int charIndex = 0;
+        while (true) {
+          if (currentNode == null) break;
+          int charComp = compareCharsAlphabetically(key.charAt(charIndex),
+                  currentNode.splitchar);
+          if (charComp == 0) {
+            charIndex++;
+            if (charIndex == key.length()) {
+              node = currentNode;
+              break;
+            }
+            currentNode = currentNode.relatives[TSTNode.EQKID];
+          } else if (charComp < 0) {
+            currentNode = currentNode.relatives[TSTNode.LOKID];
+          } else {
+            currentNode = currentNode.relatives[TSTNode.HIKID];
+          }
+        }
+        Float occur2 = null;
+        if (node != null) occur2 = ((Float) (node.data));
+        if (occur2 != null) {
+          occur += occur2.floatValue();
+        }
+        currentNode = getOrCreateNode(word.trim().toLowerCase());
+        currentNode.data = occur;
+      }
+    }
+    in.close();
+  }
+
+  /**
+   * Deletes the node passed in as an argument. If this node has non-null data,
+   * then both the node and the data will be deleted. It also deletes any other
+   * nodes in the trie that are no longer needed after the deletion of the node.
+   * 
+   *@param nodeToDelete
+   *          The node to delete.
+   */
+  private void deleteNode(TSTNode nodeToDelete) {
+    if (nodeToDelete == null) {
+      return;
+    }
+    nodeToDelete.data = null;
+    while (nodeToDelete != null) {
+      nodeToDelete = deleteNodeRecursion(nodeToDelete);
+      // deleteNodeRecursion(nodeToDelete);
+    }
+  }
+
+  /**
+   * Recursively visits each node to be deleted.
+   * 
+   * To delete a node, first set its data to null, then pass it into this
+   * method, then pass the node returned by this method into this method (make
+   * sure you don't delete the data of any of the nodes returned from this
+   * method!) and continue in this fashion until the node returned by this
+   * method is <code>null</code>.
+   * 
+   * The TSTNode instance returned by this method will be next node to be
+   * operated on by <code>deleteNodeRecursion</code> (This emulates recursive
+   * method call while avoiding the JVM overhead normally associated with a
+   * recursive method.)
+   * 
+   *@param currentNode
+   *          The node to delete.
+   *@return The next node to be called in deleteNodeRecursion.
+   */
+  private TSTNode deleteNodeRecursion(TSTNode currentNode) {
+    if (currentNode == null) {
+      return null;
+    }
+    if (currentNode.relatives[TSTNode.EQKID] != null
+            || currentNode.data != null) {
+      return null;
+    }
+    // can't delete this node if it has a non-null eq kid or data
+    TSTNode currentParent = currentNode.relatives[TSTNode.PARENT];
+    boolean lokidNull = currentNode.relatives[TSTNode.LOKID] == null;
+    boolean hikidNull = currentNode.relatives[TSTNode.HIKID] == null;
+    int childType;
+    if (currentParent.relatives[TSTNode.LOKID] == currentNode) {
+      childType = TSTNode.LOKID;
+    } else if (currentParent.relatives[TSTNode.EQKID] == currentNode) {
+      childType = TSTNode.EQKID;
+    } else if (currentParent.relatives[TSTNode.HIKID] == currentNode) {
+      childType = TSTNode.HIKID;
+    } else {
+      rootNode = null;
+      return null;
+    }
+    if (lokidNull && hikidNull) {
+      currentParent.relatives[childType] = null;
+      return currentParent;
+    }
+    if (lokidNull) {
+      currentParent.relatives[childType] = currentNode.relatives[TSTNode.HIKID];
+      currentNode.relatives[TSTNode.HIKID].relatives[TSTNode.PARENT] = currentParent;
+      return currentParent;
+    }
+    if (hikidNull) {
+      currentParent.relatives[childType] = currentNode.relatives[TSTNode.LOKID];
+      currentNode.relatives[TSTNode.LOKID].relatives[TSTNode.PARENT] = currentParent;
+      return currentParent;
+    }
+    int deltaHi = currentNode.relatives[TSTNode.HIKID].splitchar
+            - currentNode.splitchar;
+    int deltaLo = currentNode.splitchar
+            - currentNode.relatives[TSTNode.LOKID].splitchar;
+    int movingKid;
+    TSTNode targetNode;
+    if (deltaHi == deltaLo) {
+      if (Math.random() < 0.5) {
+        deltaHi++;
+      } else {
+        deltaLo++;
+      }
+    }
+    if (deltaHi > deltaLo) {
+      movingKid = TSTNode.HIKID;
+      targetNode = currentNode.relatives[TSTNode.LOKID];
+    } else {
+      movingKid = TSTNode.LOKID;
+      targetNode = currentNode.relatives[TSTNode.HIKID];
+    }
+    while (targetNode.relatives[movingKid] != null) {
+      targetNode = targetNode.relatives[movingKid];
+    }
+    targetNode.relatives[movingKid] = currentNode.relatives[movingKid];
+    currentParent.relatives[childType] = targetNode;
+    targetNode.relatives[TSTNode.PARENT] = currentParent;
+    if (!lokidNull) {
+      currentNode.relatives[TSTNode.LOKID] = null;
+    }
+    if (!hikidNull) {
+      currentNode.relatives[TSTNode.HIKID] = null;
+    }
+    return currentParent;
+  }
+
+  /**
+   * Retrieve the object indexed by a key.
+   * 
+   *@param key
+   *          A <code>String</code> index.
+   *@return The object retrieved from the Ternary Search Trie.
+   */
+  public Object get(String key) {
+    TSTNode node = getNode(key.trim().toLowerCase());
+    if (node == null) {
+      return null;
+    }
+    return node.data;
+  }
+
+  /**
+   * Retrieve the <code>Float</code> indexed by key, increment it by one unit
+   * and store the new <code>Float</code>.
+   * 
+   *@param key
+   *          A <code>String</code> index.
+   *@return The <code>Float</code> retrieved from the Ternary Search Trie.
+   */
+  public Float getAndIncrement(String key) {
+    String key2 = key.trim().toLowerCase();
+    TSTNode node = getNode(key2);
+    if (node == null) {
+      return null;
+    }
+    Float aux = (Float) (node.data);
+    if (aux == null) {
+      aux = new Float(1);
+    } else {
+      aux = new Float(aux.intValue() + 1);
+    }
+    put(key2, aux);
+    return aux;
+  }
+
+  /**
+   * Returns the key that indexes the node argument.
+   * 
+   *@param node
+   *          The node whose index is to be calculated.
+   *@return The <code>String</code> that indexes the node argument.
+   */
+  protected String getKey(TSTNode node) {
+    StringBuffer getKeyBuffer = new StringBuffer();
+    getKeyBuffer.setLength(0);
+    getKeyBuffer.append("" + node.splitchar);
+    TSTNode currentNode;
+    TSTNode lastNode;
+    currentNode = node.relatives[TSTNode.PARENT];
+    lastNode = node;
+    while (currentNode != null) {
+      if (currentNode.relatives[TSTNode.EQKID] == lastNode) {
+        getKeyBuffer.append("" + currentNode.splitchar);
+      }
+      lastNode = currentNode;
+      currentNode = currentNode.relatives[TSTNode.PARENT];
+    }
+    getKeyBuffer.reverse();
+    return getKeyBuffer.toString();
+  }
+
+  /**
+   * Returns the node indexed by key, or <code>null</code> if that node doesn't
+   * exist. Search begins at root node.
+   * 
+   *@param key
+   *          A <code>String</code> that indexes the node that is returned.
+   *@return The node object indexed by key. This object is an instance of an
+   *         inner class named <code>TernarySearchTrie.TSTNode</code>.
+   */
+  public TSTNode getNode(String key) {
+    return getNode(key, rootNode);
+  }
+
+  /**
+   * Returns the node indexed by key, or <code>null</code> if that node doesn't
+   * exist. The search begins at root node.
+   * 
+   *@param key2
+   *          A <code>String</code> that indexes the node that is returned.
+   *@param startNode
+   *          The top node defining the subtrie to be searched.
+   *@return The node object indexed by key. This object is an instance of an
+   *         inner class named <code>TernarySearchTrie.TSTNode</code>.
+   */
+  protected TSTNode getNode(String key2, TSTNode startNode) {
+    String key = key2.trim().toLowerCase();
+    if (key == null || startNode == null || key.length() == 0) {
+      return null;
+    }
+    TSTNode currentNode = startNode;
+    int charIndex = 0;
+    while (true) {
+      if (currentNode == null) {
+        return null;
+      }
+      int charComp = compareCharsAlphabetically(key.charAt(charIndex),
+              currentNode.splitchar);
+      if (charComp == 0) {
+        charIndex++;
+        if (charIndex == key.length()) {
+          return currentNode;
+        }
+        currentNode = currentNode.relatives[TSTNode.EQKID];
+      } else if (charComp < 0) {
+        currentNode = currentNode.relatives[TSTNode.LOKID];
+      } else {
+        currentNode = currentNode.relatives[TSTNode.HIKID];
+      }
+    }
+  }
+
+  /**
+   * Returns the node indexed by key, creating that node if it doesn't exist,
+   * and creating any required intermediate nodes if they don't exist.
+   * 
+   *@param key
+   *          A <code>String</code> that indexes the node that is returned.
+   *@return The node object indexed by key. This object is an instance of an
+   *         inner class named <code>TernarySearchTrie.TSTNode</code>.
+   *@exception NullPointerException
+   *              If the key is <code>null</code>.
+   *@exception IllegalArgumentException
+   *              If the key is an empty <code>String</code>.
+   */
+  protected TSTNode getOrCreateNode(String key) throws NullPointerException,
+          IllegalArgumentException {
+    if (key == null) {
+      throw new NullPointerException(
+              "attempt to get or create node with null key");
+    }
+    if (key.length() == 0) {
+      throw new IllegalArgumentException(
+              "attempt to get or create node with key of zero length");
+    }
+    if (rootNode == null) {
+      rootNode = new TSTNode(key.charAt(0), null);
+    }
+    TSTNode currentNode = rootNode;
+    int charIndex = 0;
+    while (true) {
+      int charComp = compareCharsAlphabetically(key.charAt(charIndex),
+              currentNode.splitchar);
+      if (charComp == 0) {
+        charIndex++;
+        if (charIndex == key.length()) {
+          return currentNode;
+        }
+        if (currentNode.relatives[TSTNode.EQKID] == null) {
+          currentNode.relatives[TSTNode.EQKID] = new TSTNode(key
+                  .charAt(charIndex), currentNode);
+        }
+        currentNode = currentNode.relatives[TSTNode.EQKID];
+      } else if (charComp < 0) {
+        if (currentNode.relatives[TSTNode.LOKID] == null) {
+          currentNode.relatives[TSTNode.LOKID] = new TSTNode(key
+                  .charAt(charIndex), currentNode);
+        }
+        currentNode = currentNode.relatives[TSTNode.LOKID];
+      } else {
+        if (currentNode.relatives[TSTNode.HIKID] == null) {
+          currentNode.relatives[TSTNode.HIKID] = new TSTNode(key
+                  .charAt(charIndex), currentNode);
+        }
+        currentNode = currentNode.relatives[TSTNode.HIKID];
+      }
+    }
+  }
+
+  /**
+   * Returns a <code>List</code> of keys that almost match the argument key.
+   * Keys returned will have exactly diff characters that do not match the
+   * target key, where diff is equal to the last value passed in as an argument
+   * to the <code>setMatchAlmostDiff</code> method.
+   * <p>
+   * If the <code>matchAlmost</code> method is called before the
+   * <code>setMatchAlmostDiff</code> method has been called for the first time,
+   * then diff = 0.
+   * 
+   *@param key
+   *          The target key.
+   *@return A <code>List</code> with the results.
+   */
+  public List<String> matchAlmost(String key) {
+    return matchAlmost(key, defaultNumReturnValues);
+  }
+
+  /**
+   * Returns a <code>List</code> of keys that almost match the argument key.
+   * Keys returned will have exactly diff characters that do not match the
+   * target key, where diff is equal to the last value passed in as an argument
+   * to the <code>setMatchAlmostDiff</code> method.
+   * <p>
+   * If the <code>matchAlmost</code> method is called before the
+   * <code>setMatchAlmostDiff</code> method has been called for the first time,
+   * then diff = 0.
+   * 
+   *@param key
+   *          The target key.
+   *@param numReturnValues
+   *          The maximum number of values returned by this method.
+   *@return A <code>List</code> with the results
+   */
+  public List<String> matchAlmost(String key, int numReturnValues) {
+    return matchAlmostRecursion(rootNode, 0, matchAlmostDiff, key,
+            ((numReturnValues < 0) ? -1 : numReturnValues), new Vector<String>(), false);
+  }
+
+  /**
+   * Recursivelly vists the nodes in order to find the ones that almost match a
+   * given key.
+   * 
+   *@param currentNode
+   *          The current node.
+   *@param charIndex
+   *          The current char.
+   *@param d
+   *          The number of differences so far.
+   *@param matchAlmostNumReturnValues
+   *          The maximum number of values in the result <code>List</code>.
+   *@param matchAlmostResult2
+   *          The results so far.
+   *@param upTo
+   *          If true all keys having up to and including matchAlmostDiff
+   *          mismatched letters will be included in the result (including a key
+   *          that is exactly the same as the target string) otherwise keys will
+   *          be included in the result only if they have exactly
+   *          matchAlmostDiff number of mismatched letters.
+   *@param matchAlmostKey
+   *          The key being searched.
+   *@return A <code>List</code> with the results.
+   */
+  private List<String> matchAlmostRecursion(TSTNode currentNode, int charIndex,
+          int d, String matchAlmostKey, int matchAlmostNumReturnValues,
+          List<String> matchAlmostResult2, boolean upTo) {
+    if ((currentNode == null)
+            || (matchAlmostNumReturnValues != -1 && matchAlmostResult2.size() >= matchAlmostNumReturnValues)
+            || (d < 0) || (charIndex >= matchAlmostKey.length())) {
+      return matchAlmostResult2;
+    }
+    int charComp = compareCharsAlphabetically(matchAlmostKey.charAt(charIndex),
+            currentNode.splitchar);
+    List<String> matchAlmostResult = matchAlmostResult2;
+    if ((d > 0) || (charComp < 0)) {
+      matchAlmostResult = matchAlmostRecursion(
+              currentNode.relatives[TSTNode.LOKID], charIndex, d,
+              matchAlmostKey, matchAlmostNumReturnValues, matchAlmostResult,
+              upTo);
+    }
+    int nextD = (charComp == 0) ? d : d - 1;
+    boolean cond = (upTo) ? (nextD >= 0) : (nextD == 0);
+    if ((matchAlmostKey.length() == charIndex + 1) && cond
+            && (currentNode.data != null)) {
+      matchAlmostResult.add(getKey(currentNode));
+    }
+    matchAlmostResult = matchAlmostRecursion(
+            currentNode.relatives[TSTNode.EQKID], charIndex + 1, nextD,
+            matchAlmostKey, matchAlmostNumReturnValues, matchAlmostResult, upTo);
+    if ((d > 0) || (charComp > 0)) {
+      matchAlmostResult = matchAlmostRecursion(
+              currentNode.relatives[TSTNode.HIKID], charIndex, d,
+              matchAlmostKey, matchAlmostNumReturnValues, matchAlmostResult,
+              upTo);
+    }
+    return matchAlmostResult;
+  }
+
+  /**
+   * Returns an alphabetical <code>List</code> of all keys in the trie that
+   * begin with a given prefix. Only keys for nodes having non-null data are
+   * included in the <code>List</code>.
+   * 
+   *@param prefix
+   *          Each key returned from this method will begin with the characters
+   *          in prefix.
+   *@return A <code>List</code> with the results.
+   */
+  public List<String> matchPrefix(String prefix) {
+    return matchPrefix(prefix, defaultNumReturnValues);
+  }
+
+  /**
+   * Returns an alphabetical <code>List</code> of all keys in the trie that
+   * begin with a given prefix. Only keys for nodes having non-null data are
+   * included in the <code>List</code>.
+   * 
+   *@param prefix
+   *          Each key returned from this method will begin with the characters
+   *          in prefix.
+   *@param numReturnValues
+   *          The maximum number of values returned from this method.
+   *@return A <code>List</code> with the results
+   */
+  public List<String> matchPrefix(String prefix, int numReturnValues) {
+    Vector<String> sortKeysResult = new Vector<String>();
+    TSTNode startNode = getNode(prefix);
+    if (startNode == null) {
+      return sortKeysResult;
+    }
+    if (startNode.data != null) {
+      sortKeysResult.addElement(getKey(startNode));
+    }
+    return sortKeysRecursion(startNode.relatives[TSTNode.EQKID],
+            ((numReturnValues < 0) ? -1 : numReturnValues), sortKeysResult);
+  }
+
+  /**
+   * Returns the number of nodes in the trie that have non-null data.
+   * 
+   *@return The number of nodes in the trie that have non-null data.
+   */
+  public int numDataNodes() {
+    return numDataNodes(rootNode);
+  }
+
+  /**
+   * Returns the number of nodes in the subtrie below and including the starting
+   * node. The method counts only nodes that have non-null data.
+   * 
+   *@param startingNode
+   *          The top node of the subtrie. the node that defines the subtrie.
+   *@return The total number of nodes in the subtrie.
+   */
+  protected int numDataNodes(TSTNode startingNode) {
+    return recursiveNodeCalculator(startingNode, true, 0);
+  }
+
+  /**
+   * Returns the total number of nodes in the trie. The method counts nodes
+   * whether or not they have data.
+   * 
+   *@return The total number of nodes in the trie.
+   */
+  public int numNodes() {
+    return numNodes(rootNode);
+  }
+
+  /**
+   * Returns the total number of nodes in the subtrie below and including the
+   * starting Node. The method counts nodes whether or not they have data.
+   * 
+   *@param startingNode
+   *          The top node of the subtrie. The node that defines the subtrie.
+   *@return The total number of nodes in the subtrie.
+   */
+  protected int numNodes(TSTNode startingNode) {
+    return recursiveNodeCalculator(startingNode, false, 0);
+  }
+
+  /**
+   * Stores a value in the trie. The value may be retrieved using the key.
+   * 
+   *@param key
+   *          A <code>String</code> that indexes the object to be stored.
+   *@param value
+   *          The object to be stored in the Trie.
+   */
+  public void put(String key, Object value) {
+    getOrCreateNode(key.trim().toLowerCase()).data = value;
+  }
+
+  /**
+   * Recursivelly visists each node to calculate the number of nodes.
+   * 
+   *@param currentNode
+   *          The current node.
+   *@param checkData
+   *          If true we check the data to be different of <code>null</code>.
+   *@param numNodes2
+   *          The number of nodes so far.
+   *@return The number of nodes accounted.
+   */
+  private int recursiveNodeCalculator(TSTNode currentNode, boolean checkData,
+          int numNodes2) {
+    if (currentNode == null) {
+      return numNodes2;
+    }
+    int numNodes = recursiveNodeCalculator(
+            currentNode.relatives[TSTNode.LOKID], checkData, numNodes2);
+    numNodes = recursiveNodeCalculator(currentNode.relatives[TSTNode.EQKID],
+            checkData, numNodes);
+    numNodes = recursiveNodeCalculator(currentNode.relatives[TSTNode.HIKID],
+            checkData, numNodes);
+    if (checkData) {
+      if (currentNode.data != null) {
+        numNodes++;
+      }
+    } else {
+      numNodes++;
+    }
+    return numNodes;
+  }
+
+  /**
+   * Removes the value indexed by key. Also removes all nodes that are rendered
+   * unnecessary by the removal of this data.
+   * 
+   *@param key
+   *          A <code>string</code> that indexes the object to be removed from
+   *          the Trie.
+   */
+  public void remove(String key) {
+    deleteNode(getNode(key.trim().toLowerCase()));
+  }
+
+  /**
+   * Sets the number of characters by which words can differ from target word
+   * when calling the <code>matchAlmost</code> method.
+   * <p>
+   * Arguments less than 0 will set the char difference to 0, and arguments
+   * greater than 3 will set the char difference to 3.
+   * 
+   *@param diff
+   *          The number of characters by which words can differ from target
+   *          word.
+   */
+  public void setMatchAlmostDiff(int diff) {
+    if (diff < 0) {
+      matchAlmostDiff = 0;
+    } else if (diff > 3) {
+      matchAlmostDiff = 3;
+    } else {
+      matchAlmostDiff = diff;
+    }
+  }
+
+  /**
+   * Sets the default maximum number of values returned from the
+   * <code>matchPrefix</code> and <code>matchAlmost</code> methods.
+   * <p>
+   * The value should be set this to -1 to get an unlimited number of return
+   * values. note that the methods mentioned above provide overloaded versions
+   * that allow you to specify the maximum number of return values, in which
+   * case this value is temporarily overridden.
+   * 
+   **@param num
+   *          The number of values that will be returned when calling the
+   *          methods above.
+   */
+  public void setNumReturnValues(int num) {
+    defaultNumReturnValues = (num < 0) ? -1 : num;
+  }
+
+  /**
+   * Returns keys sorted in alphabetical order. This includes the start Node and
+   * all nodes connected to the start Node.
+   * <p>
+   * The number of keys returned is limited to numReturnValues. To get a list
+   * that isn't limited in size, set numReturnValues to -1.
+   * 
+   *@param startNode
+   *          The top node defining the subtrie to be searched.
+   *@param numReturnValues
+   *          The maximum number of values returned from this method.
+   *@return A <code>List</code> with the results.
+   */
+  protected List<String> sortKeys(TSTNode startNode, int numReturnValues) {
+    return sortKeysRecursion(startNode, ((numReturnValues < 0) ? -1
+            : numReturnValues), new Vector<String>());
+  }
+
+  /**
+   * Returns keys sorted in alphabetical order. This includes the current Node
+   * and all nodes connected to the current Node.
+   * <p>
+   * Sorted keys will be appended to the end of the resulting <code>List</code>.
+   * The result may be empty when this method is invoked, but may not be
+   * <code>null</code>.
+   * 
+   *@param currentNode
+   *          The current node.
+   *@param sortKeysNumReturnValues
+   *          The maximum number of values in the result.
+   *@param sortKeysResult2
+   *          The results so far.
+   *@return A <code>List</code> with the results.
+   */
+  private List<String> sortKeysRecursion(TSTNode currentNode,
+          int sortKeysNumReturnValues, List<String> sortKeysResult2) {
+    if (currentNode == null) {
+      return sortKeysResult2;
+    }
+    List<String> sortKeysResult = sortKeysRecursion(
+            currentNode.relatives[TSTNode.LOKID], sortKeysNumReturnValues,
+            sortKeysResult2);
+    if (sortKeysNumReturnValues != -1
+            && sortKeysResult.size() >= sortKeysNumReturnValues) {
+      return sortKeysResult;
+    }
+    if (currentNode.data != null) {
+      sortKeysResult.add(getKey(currentNode));
+    }
+    sortKeysResult = sortKeysRecursion(currentNode.relatives[TSTNode.EQKID],
+            sortKeysNumReturnValues, sortKeysResult);
+    return sortKeysRecursion(currentNode.relatives[TSTNode.HIKID],
+            sortKeysNumReturnValues, sortKeysResult);
+  }
+
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,142 @@
+package org.apache.solr.spelling.suggest.tst;
+
+import java.util.*;
+
+public class TSTAutocomplete {
+
+  /**
+   * Inserting keys in TST in the order middle,small,big (lexicographic measure)
+   * recursively creates a balanced tree which reduces insertion and search
+   * times significantly.
+   * 
+   * @param tokens
+   *          Sorted list of keys to be inserted in TST.
+   * @param lo
+   *          stores the lower index of current list.
+   * @param hi
+   *          stores the higher index of current list.
+   * @param root
+   *          a reference object to root of TST.
+   */
+  public void balancedTree(Object[] tokens, Object[] vals, int lo, int hi,
+          TernaryTreeNode root) {
+    if (lo > hi) return;
+    int mid = (lo + hi) / 2;
+    root = insert(root, (String) tokens[mid], vals[mid], 0);
+    balancedTree(tokens, vals, lo, mid - 1, root);
+    balancedTree(tokens, vals, mid + 1, hi, root);
+  }
+
+  /**
+   * Inserts a key in TST creating a series of Binary Search Trees at each node.
+   * The key is actually stored across the eqKid of each node in a successive
+   * manner.
+   * 
+   * @param currentNode
+   *          a reference node where the insertion will take currently.
+   * @param s
+   *          key to be inserted in TST.
+   * @param x
+   *          index of character in key to be inserted currently.
+   * @return currentNode The new reference to root node of TST
+   */
+  public TernaryTreeNode insert(TernaryTreeNode currentNode, String s,
+          Object val, int x) {
+    if (s == null || s.length() <= x) {
+      return currentNode;
+    }
+    if (currentNode == null) {
+      TernaryTreeNode newNode = new TernaryTreeNode();
+      newNode.splitchar = s.charAt(x);
+      currentNode = newNode;
+      if (x < s.length() - 1) {
+        currentNode.eqKid = insert(currentNode.eqKid, s, val, x + 1);
+      } else {
+        currentNode.token = s;
+        currentNode.val = val;
+        return currentNode;
+      }
+    } else if (currentNode.splitchar > s.charAt(x)) {
+      currentNode.loKid = insert(currentNode.loKid, s, val, x);
+    } else if (currentNode.splitchar == s.charAt(x)) {
+      if (x < s.length() - 1) {
+        currentNode.eqKid = insert(currentNode.eqKid, s, val, x + 1);
+      } else {
+        currentNode.token = s;
+        currentNode.val = val;
+        return currentNode;
+      }
+    } else {
+      currentNode.hiKid = insert(currentNode.hiKid, s, val, x);
+    }
+    return currentNode;
+  }
+
+  /**
+   * Auto-completes a given prefix query using Depth-First Search with the end
+   * of prefix as source node each time finding a new leaf to get a complete key
+   * to be added in the suggest list.
+   * 
+   * @param root
+   *          a reference to root node of TST.
+   * @param s
+   *          prefix query to be auto-completed.
+   * @param x
+   *          index of current character to be searched while traversing through
+   *          the prefix in TST.
+   * @return suggest list of auto-completed keys for the given prefix query.
+   */
+  public ArrayList<TernaryTreeNode> prefixCompletion(TernaryTreeNode root,
+          String s, int x) {
+
+    TernaryTreeNode p = root;
+    ArrayList<TernaryTreeNode> suggest = new ArrayList<TernaryTreeNode>();
+
+    while (p != null) {
+      if (s.charAt(x) < p.splitchar) {
+        p = p.loKid;
+      } else if (s.charAt(x) == p.splitchar) {
+        if (x == s.length() - 1) {
+          break;
+        } else {
+          x++;
+        }
+        p = p.eqKid;
+      } else {
+        p = p.hiKid;
+      }
+    }
+
+    if (p == null) return suggest;
+    if (p.eqKid == null && p.token == null) return suggest;
+    if (p.eqKid == null && p.token != null) {
+      suggest.add(p);
+      return suggest;
+    }
+
+    if (p.token != null) {
+      suggest.add(p);
+    }
+    p = p.eqKid;
+
+    Stack<TernaryTreeNode> st = new Stack<TernaryTreeNode>();
+    st.push(p);
+    while (!st.empty()) {
+      TernaryTreeNode top = (TernaryTreeNode) st.peek();
+      st.pop();
+      if (top.token != null) {
+        suggest.add(top);
+      }
+      if (top.eqKid != null) {
+        st.push(top.eqKid);
+      }
+      if (top.loKid != null) {
+        st.push(top.loKid);
+      }
+      if (top.hiKid != null) {
+        st.push(top.hiKid);
+      }
+    }
+    return suggest;
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTAutocomplete.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,89 @@
+package org.apache.solr.spelling.suggest.tst;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.spelling.suggest.Lookup;
+import org.apache.solr.spelling.suggest.SortedTermFreqIteratorWrapper;
+import org.apache.solr.util.SortedIterator;
+import org.apache.solr.util.TermFreqIterator;
+
+public class TSTLookup extends Lookup {
+  TernaryTreeNode root;
+  TSTAutocomplete autocomplete;
+
+  @Override
+  public void init(NamedList config, SolrCore core) {
+  }
+
+  @Override
+  public void build(TermFreqIterator tfit) throws IOException {
+    root = new TernaryTreeNode();
+    autocomplete = new TSTAutocomplete();
+    // buffer first
+    if (!(tfit instanceof SortedIterator)) {
+      // make sure it's sorted
+      tfit = new SortedTermFreqIteratorWrapper(tfit);
+    }
+
+    ArrayList<String> tokens = new ArrayList<String>();
+    ArrayList<Float> vals = new ArrayList<Float>();
+    while (tfit.hasNext()) {
+      tokens.add(tfit.next());
+      vals.add(new Float(tfit.freq()));
+    }
+    autocomplete.balancedTree(tokens.toArray(), vals.toArray(), 0, tokens.size() - 1, root);
+  }
+
+  @Override
+  public boolean add(String key, Object value) {
+    autocomplete.insert(root, key, value, 0);
+    // XXX we don't know if a new node was created
+    return true;
+  }
+
+  @Override
+  public Object get(String key) {
+    throw new UnsupportedOperationException("get() is not supported here");
+  }
+
+  @Override
+  public List<LookupResult> lookup(String key, boolean onlyMorePopular, int num) {
+    List<TernaryTreeNode> list = autocomplete.prefixCompletion(root, key, 0);
+    List<LookupResult> res = new ArrayList<LookupResult>();
+    if (list == null || list.size() == 0) {
+      return res;
+    }
+    int maxCnt = Math.min(num, list.size());
+    if (onlyMorePopular) {
+      LookupPriorityQueue queue = new LookupPriorityQueue(num);
+      for (TernaryTreeNode ttn : list) {
+        queue.insertWithOverflow(new LookupResult(ttn.token, (Float)ttn.val));
+      }
+      for (LookupResult lr : queue.getResults()) {
+        res.add(lr);
+      }
+    } else {
+      for (int i = 0; i < maxCnt; i++) {
+        TernaryTreeNode ttn = list.get(i);
+        res.add(new LookupResult(ttn.token, (Float)ttn.val));
+      }
+    }
+    return res;
+  }
+
+  @Override
+  public boolean load(File storeDir) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean store(File storeDir) throws IOException {
+    return false;
+  }
+
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,21 @@
+package org.apache.solr.spelling.suggest.tst;
+
+/**
+ * The class creates a TST node.
+ * @variable splitchar the character stored by a node.
+ * @variable loKid a reference object to the node containing character smaller than
+ * this node's character.
+ * @variable eqKid a reference object to the node containg character next to this
+ * node's character as occuring in the inserted token.
+ * @variable hiKid a reference object to the node containing character higher than
+ * this node's character.
+ * @variable token used by leaf nodes to store the complete tokens to be added to 
+ * suggest list while auto-completing the prefix.
+ */
+
+public class TernaryTreeNode {
+	char splitchar;
+	TernaryTreeNode loKid, eqKid, hiKid;
+	String token;
+	Object val;
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/spelling/suggest/tst/TernaryTreeNode.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/util/HighFrequencyDictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/util/HighFrequencyDictionary.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/util/HighFrequencyDictionary.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/util/HighFrequencyDictionary.java Mon Aug 23 13:44:00 2010
@@ -53,7 +53,7 @@ public class HighFrequencyDictionary imp
     return new HighFrequencyIterator();
   }
 
-  final class HighFrequencyIterator implements Iterator {
+  final class HighFrequencyIterator implements TermFreqIterator, SortedIterator {
     private TermsEnum termsEnum;
     private BytesRef actualTerm;
     private boolean hasNextCalled;
@@ -74,8 +74,12 @@ public class HighFrequencyDictionary imp
     private boolean isFrequent(int freq) {
       return freq >= minNumDocs;
     }
-
-    public Object next() {
+    
+    public float freq() {
+      return termsEnum.docFreq();
+    }
+    
+    public String next() {
       if (!hasNextCalled && !hasNext()) {
         return null;
       }

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,11 @@
+package org.apache.solr.util;
+
+import java.util.Iterator;
+
+/**
+ * Marker interface to signal that elements coming from {@link Iterator}
+ * come in ascending lexicographic order.
+ */
+public interface SortedIterator {
+
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/util/SortedIterator.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java (added)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,37 @@
+package org.apache.solr.util;
+
+import java.util.Iterator;
+
+public interface TermFreqIterator extends Iterator<String> {
+
+  public float freq();
+  
+  public static class TermFreqIteratorWrapper implements TermFreqIterator {
+    private Iterator wrapped;
+    
+    public TermFreqIteratorWrapper(Iterator wrapped) {
+      this.wrapped = wrapped;
+    }
+
+    @Override
+    public float freq() {
+      return 1.0f;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return wrapped.hasNext();
+    }
+
+    @Override
+    public String next() {
+      return wrapped.next().toString();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+}

Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/util/TermFreqIterator.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DummyCustomParamSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DummyCustomParamSpellChecker.java?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DummyCustomParamSpellChecker.java (original)
+++ lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DummyCustomParamSpellChecker.java Mon Aug 23 13:44:00 2010
@@ -35,7 +35,7 @@ import java.util.Iterator;
 public class DummyCustomParamSpellChecker extends SolrSpellChecker {
 
   @Override
-  public void reload() throws IOException {
+  public void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException {
 
   }
 

Added: lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java?rev=988120&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java (added)
+++ lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java Mon Aug 23 13:44:00 2010
@@ -0,0 +1,253 @@
+package org.apache.solr.spelling.suggest;
+
+import java.io.StringWriter;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SpellingParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.QueryResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.spelling.suggest.Lookup.LookupResult;
+import org.apache.solr.spelling.suggest.jaspell.JaspellLookup;
+import org.apache.solr.spelling.suggest.tst.TSTLookup;
+import org.apache.solr.util.AbstractSolrTestCase;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.TermFreqIterator;
+import org.apache.solr.util.TestHarness;
+
+public class SuggesterTest extends AbstractSolrTestCase {
+  SolrRequestHandler handler;
+
+  @Override
+  public String getSchemaFile() {
+    return "schema-spellchecker.xml";
+  }
+
+  @Override
+  public String getSolrConfigFile() {
+    return "solrconfig-spellchecker.xml";
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // empty
+    h.validateUpdate("<delete><query>*:*</query></delete>");
+    // populate
+    h.validateAddDoc(
+            "id", "1",
+            "text", "acceptable accidentally accommodate acquire"
+            );
+    h.validateAddDoc(
+            "id", "2",
+            "text", "believe bellwether accommodate acquire"
+            );
+    h.validateAddDoc(
+            "id", "3",
+            "text", "cemetery changeable conscientious consensus acquire bellwether"
+            );
+    h.validateUpdate("<commit/>");
+    handler = h.getCore().getRequestHandler("/suggest");
+    // build
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(SpellingParams.SPELLCHECK_BUILD, true);
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    handler.handleRequest(req, rsp);
+  }
+  
+  private String assertXPath(SolrCore core, SolrQueryRequest req, SolrQueryResponse rsp, String... tests) throws Exception {
+    StringWriter sw = new StringWriter(32000);
+    QueryResponseWriter responseWriter = core.getQueryResponseWriter(req);
+    responseWriter.write(sw,req,rsp);
+    req.close();
+    System.out.println(sw.toString());
+    return h.validateXPath(sw.toString(), tests);
+  }
+
+  public void testSuggestions() throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(CommonParams.Q, "ac");
+    params.set(SpellingParams.SPELLCHECK_COUNT, 2);
+    params.set(SpellingParams.SPELLCHECK_ONLY_MORE_POPULAR, true);
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    handler.handleRequest(req, rsp);
+    String res = assertXPath(h.getCore(), req, rsp, 
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/int[@name='numFound'][.='2']",
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/arr[@name='suggestion']/str[1][.='acquire']",
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/arr[@name='suggestion']/str[2][.='accommodate']"
+            );
+    assertNull(res, res);
+  }
+  
+  public void testReload() throws Exception {
+    String coreName = h.getCore().getName();
+    RefCounted<SolrIndexSearcher> searcher = h.getCore().getSearcher();
+    SolrIndexSearcher indexSearcher = searcher.get();
+    log.info("Core " + coreName + ", NumDocs before reload: " + indexSearcher.getIndexReader().numDocs());
+    log.info("Directory: " + indexSearcher.getIndexDir());
+    searcher.decref();
+    h.close();
+    solrConfig = TestHarness.createConfig(getSolrConfigFile());
+    h = new TestHarness( dataDir.getAbsolutePath(),
+            solrConfig,
+            getSchemaFile());
+    searcher = h.getCore().getSearcher();
+    indexSearcher = searcher.get();
+    log.info("Core " + coreName + ", NumDocs now: " + indexSearcher.getIndexReader().numDocs());
+    log.info("Directory: " + indexSearcher.getIndexDir());
+    searcher.decref();
+    // rebuilds on commit
+    h.validateUpdate("<commit/>");
+    handler = h.getCore().getRequestHandler("/suggest");
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(CommonParams.Q, "ac");
+    params.set(SpellingParams.SPELLCHECK_COUNT, 2);
+    params.set(SpellingParams.SPELLCHECK_ONLY_MORE_POPULAR, true);
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), params);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    handler.handleRequest(req, rsp);
+    String res = assertXPath(h.getCore(), req, rsp, 
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/int[@name='numFound'][.='2']",
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/arr[@name='suggestion']/str[1][.='acquire']",
+            "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='ac']/arr[@name='suggestion']/str[2][.='accommodate']"
+            );
+    assertNull(res, res);
+  }
+  
+  private TermFreqIterator getTFIT() {
+    final int count = 100000;
+    TermFreqIterator tfit = new TermFreqIterator() {
+      Random r = new Random(1234567890L);
+      Random r1 = new Random(1234567890L);
+      int pos;
+
+      @Override
+      public float freq() {
+        return r1.nextInt(4);
+      }
+
+      @Override
+      public boolean hasNext() {
+        return pos < count;
+      }
+
+      @Override
+      public String next() {
+        pos++;
+        return Long.toString(r.nextLong());
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+      
+    };
+    return tfit;
+  }
+  
+  private void _benchmark(Lookup lookup, Map<String,Integer> ref, boolean estimate, Bench bench) throws Exception {
+    long start = System.currentTimeMillis();
+    lookup.build(getTFIT());
+    long buildTime = System.currentTimeMillis() - start;
+    TermFreqIterator tfit = getTFIT();
+    long elapsed = 0;
+    while (tfit.hasNext()) {
+      String key = tfit.next();
+      // take only the first part of the key
+      int len = key.length() > 4 ? key.length() / 3 : 2;
+      String prefix = key.substring(0, len);
+      start = System.nanoTime();
+      List<LookupResult> res = lookup.lookup(prefix, true, 10);
+      elapsed += System.nanoTime() - start;
+      assertTrue(res.size() > 0);
+      for (LookupResult lr : res) {
+        assertTrue(lr.key.startsWith(prefix));
+      }
+      if (ref != null) { // verify the counts
+        Integer Cnt = ref.get(key);
+        if (Cnt == null) { // first pass
+          ref.put(key, res.size());
+        } else {
+          assertEquals(key + ", prefix: " + prefix, Cnt.intValue(), res.size());
+        }
+      }
+    }
+    if (estimate) {
+      RamUsageEstimator rue = new RamUsageEstimator();
+      long size = rue.estimateRamUsage(lookup);
+      System.err.println(lookup.getClass().getSimpleName() + " - size=" + size);
+    }
+    if (bench != null) {
+      bench.buildTime += buildTime;
+      bench.lookupTime +=  elapsed;
+    }
+  }
+  
+  class Bench {
+    long buildTime;
+    long lookupTime;
+  }
+  
+  public void testBenchmark() throws Exception {
+    // this benchmark is very time consuming
+    boolean doTest = false;
+    if (!doTest) {
+      return;
+    }
+    Map<String,Integer> ref = new HashMap<String,Integer>();
+    JaspellLookup jaspell = new JaspellLookup();
+    TSTLookup tst = new TSTLookup();
+    
+    _benchmark(tst, ref, true, null);
+    _benchmark(jaspell, ref, true, null);
+    jaspell = null;
+    tst = null;
+    int count = 100;
+    Bench b = runBenchmark(JaspellLookup.class, count);
+    System.err.println(JaspellLookup.class.getSimpleName() + ": buildTime[ms]=" + (b.buildTime / count) +
+            " lookupTime[ms]=" + (b.lookupTime / count / 1000000));
+    b = runBenchmark(TSTLookup.class, count);
+    System.err.println(TSTLookup.class.getSimpleName() + ": buildTime[ms]=" + (b.buildTime / count) +
+            " lookupTime[ms]=" + (b.lookupTime / count / 1000000));
+  }
+  
+  private Bench runBenchmark(Class<? extends Lookup> cls, int count) throws Exception {
+    System.err.println("* Running " + count + " iterations for " + cls.getSimpleName() + " ...");
+    System.err.println("  - warm-up 10 iterations...");
+    for (int i = 0; i < 10; i++) {
+      System.runFinalization();
+      System.gc();
+      Lookup lookup = cls.newInstance();
+      _benchmark(lookup, null, false, null);
+      lookup = null;
+    }
+    Bench b = new Bench();
+    System.err.print("  - main iterations:"); System.err.flush();
+    for (int i = 0; i < count; i++) {
+      System.runFinalization();
+      System.gc();
+      Lookup lookup = cls.newInstance();
+      _benchmark(lookup, null, false, b);
+      lookup = null;
+      if (i > 0 && (i % 10 == 0)) {
+        System.err.print(" " + i);
+        System.err.flush();
+      }
+    }
+    System.err.println();
+    return b;
+  }
+}

Propchange: lucene/dev/trunk/solr/src/test/org/apache/solr/spelling/suggest/SuggesterTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/dev/trunk/solr/src/test/test-files/solr/conf/schema-spellchecker.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/test-files/solr/conf/schema-spellchecker.xml?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/test/test-files/solr/conf/schema-spellchecker.xml (original)
+++ lucene/dev/trunk/solr/src/test/test-files/solr/conf/schema-spellchecker.xml Mon Aug 23 13:44:00 2010
@@ -68,9 +68,13 @@
  <fields>
    <field name="id" type="string" indexed="true" stored="true"/>
    <field name="spell" type="spellText" indexed="true" stored="true" />
+   <field name="suggest" type="spellText" indexed="true" stored="true" />
    <field name="text" type="text" indexed="true" stored="false" multiValued="true"/>
  </fields>
 
+ <copyField source="text" dest="spell"/>
+ <copyField source="text" dest="suggest"/>
+
  <!-- field to use to determine and enforce document uniqueness. -->
  <uniqueKey>id</uniqueKey>
 

Modified: lucene/dev/trunk/solr/src/test/test-files/solr/conf/solrconfig-spellchecker.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/test-files/solr/conf/solrconfig-spellchecker.xml?rev=988120&r1=988119&r2=988120&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/test/test-files/solr/conf/solrconfig-spellchecker.xml (original)
+++ lucene/dev/trunk/solr/src/test/test-files/solr/conf/solrconfig-spellchecker.xml Mon Aug 23 13:44:00 2010
@@ -92,6 +92,30 @@
 
    </requestHandler>
 
+  <!-- Suggest component -->
+  <searchComponent class="solr.SpellCheckComponent" name="suggest">
+    <lst name="spellchecker">
+      <str name="name">suggest</str>
+      <str name="classname">org.apache.solr.spelling.suggest.Suggester</str>
+      <str name="lookupImpl">org.apache.solr.spelling.suggest.jaspell.JaspellLookup</str>
+      <str name="field">suggest</str>
+      <str name="buildOnCommit">true</str>
+<!--
+      <str name="sourceLocation">american-english</str>
+-->
+    </lst>
+  </searchComponent>
+  <requestHandler class="org.apache.solr.handler.component.SearchHandler" name="/suggest">
+    <lst name="defaults">
+      <str name="spellcheck">true</str>
+      <str name="spellcheck.dictionary">suggest</str>
+      <str name="spellcheck.collate">true</str>
+    </lst>
+    <arr name="components">
+      <str>suggest</str>
+    </arr>
+  </requestHandler>
+
   
 
   <queryResponseWriter name="standard" class="solr.XMLResponseWriter"/>