You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/02/19 18:18:36 UTC

svn commit: r1291023 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/contrib/ lucene/contrib/spellchecker/ lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/ lucene/contrib/spellchecker/src/test/org/apache/lucene/search/s...

Author: rmuir
Date: Sun Feb 19 17:18:35 2012
New Revision: 1291023

URL: http://svn.apache.org/viewvc?rev=1291023&view=rev
Log:
LUCENE-3714: add weighted FST suggester impl

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
      - copied unchanged from r1291020, lucene/dev/trunk/modules/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
    lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
      - copied unchanged from r1291020, lucene/dev/trunk/modules/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/spelling/suggest/fst/WFSTLookupFactory.java
      - copied unchanged from r1291020, lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/WFSTLookupFactory.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java
      - copied unchanged from r1291020, lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterWFSTTest.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
    lucene/dev/branches/branch_3x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/branches/branch_3x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
    lucene/dev/branches/branch_3x/solr/   (props changed)
    lucene/dev/branches/branch_3x/solr/CHANGES.txt
    lucene/dev/branches/branch_3x/solr/core/   (props changed)
    lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-spellchecker.xml
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Sun Feb 19 17:18:35 2012
@@ -162,6 +162,9 @@ New Features
 
 * LUCENE-3725: Added optional packing to FST building; this uses extra
   RAM during building but results in a smaller FST.  (Mike McCandless)
+
+* LUCENE-3714: Add top N shortest cost paths search for FST<Long>.
+  (Robert Muir, Dawid Weiss, Mike McCandless)
   
 Bug fixes
 

Modified: lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt Sun Feb 19 17:18:35 2012
@@ -52,6 +52,9 @@ New Features
 
  * LUCENE-3602: Added query time joining under the join contrib. (Martijn van Groningen, Michael McCandless)
   
+ * LUCENE-3714: Add WFSTCompletionLookup suggester that supports more fine-grained
+   ranking for suggestions.  (Mike McCandless, Dawid Weiss, Robert Muir)
+
 API Changes
 
  * LUCENE-3596: DirectoryTaxonomyWriter.openIndexWriter() now takes an 

Modified: lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/spellchecker/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java Sun Feb 19 17:18:35 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
-import org.apache.lucene.search.suggest.fst.FSTLookup;
+import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
 import org.apache.lucene.search.suggest.tst.TSTLookup;
 
@@ -49,7 +49,8 @@ public class LookupBenchmarkTest extends
   private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList(
       JaspellLookup.class, 
       TSTLookup.class,
-      FSTCompletionLookup.class);
+      FSTCompletionLookup.class,
+      WFSTCompletionLookup.class);
 
   private final static int rounds = 15;
   private final static int warmup = 5;
@@ -74,6 +75,7 @@ public class LookupBenchmarkTest extends
    */
   @BeforeClass
   public static void setup() throws Exception {
+    assert false : "disable assertions before running benchmarks!";
     List<TermFreq> input = readTop50KWiki();
     Collections.shuffle(input, random);
     LookupBenchmarkTest.dictionaryInput = input.toArray(new TermFreq [input.size()]);

Modified: lucene/dev/branches/branch_3x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_3x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Sun Feb 19 17:18:35 2012
@@ -234,7 +234,325 @@ public final class Util {
       }
     }    
   }
-  
+
+  private static class FSTPath implements Comparable<FSTPath> {
+    public FST.Arc<Long> arc;
+    public long cost;
+    public final IntsRef input = new IntsRef();
+
+    public FSTPath(long cost, FST.Arc<Long> arc) {
+      this.arc = new FST.Arc<Long>().copyFrom(arc);
+      this.cost = cost;
+    }
+
+    @Override
+    public String toString() {
+      return "input=" + input + " cost=" + cost;
+    }
+
+    //@Override
+    public int compareTo(FSTPath other) {
+      if (cost < other.cost) {
+        return -1;
+      } else if (cost > other.cost) {
+        return 1;
+      } else  {
+        return input.compareTo(other.input);
+      }
+    }
+  }
+
+  private static class TopNSearcher {
+
+    private final FST<Long> fst;
+    private final FST.Arc<Long> fromNode;
+    private final int topN;
+
+    // Set once the queue has filled:
+    FSTPath bottom = null;
+
+    TreeSet<FSTPath> queue = null;
+
+    public TopNSearcher(FST<Long> fst, FST.Arc<Long> fromNode, int topN) {
+      this.fst = fst;
+      this.topN = topN;
+      this.fromNode = fromNode;
+    }
+
+    // If back plus this arc is competitive then add to queue:
+    private void addIfCompetitive(FSTPath path) {
+
+      assert queue != null;
+
+      long cost = path.cost + path.arc.output;
+      //System.out.println("  addIfCompetitive bottom=" + bottom + " queue.size()=" + queue.size());
+
+      if (bottom != null) {
+
+        if (cost > bottom.cost) {
+          // Doesn't compete
+          return;
+        } else if (cost == bottom.cost) {
+          // Tie break by alpha sort on the input:
+          path.input.grow(path.input.length+1);
+          path.input.ints[path.input.length++] = path.arc.label;
+          final int cmp = bottom.input.compareTo(path.input);
+          path.input.length--;
+          assert cmp != 0;
+          if (cmp < 0) {
+            // Doesn't compete
+            return;
+          }
+        }
+        // Competes
+      } else {
+        // Queue isn't full yet, so any path we hit competes:
+      }
+
+      final FSTPath newPath = new FSTPath(cost, path.arc);
+
+      newPath.input.grow(path.input.length+1);
+      System.arraycopy(path.input.ints, 0, newPath.input.ints, 0, path.input.length);
+      newPath.input.ints[path.input.length] = path.arc.label;
+      newPath.input.length = path.input.length+1;
+
+      //System.out.println("    add path=" + newPath);
+      queue.add(newPath);
+      if (bottom != null) {
+        // only if java6 is available: see replacement below
+        // final FSTPath removed = queue.pollLast();
+        assert !queue.isEmpty();
+        final FSTPath removed = queue.last();
+        queue.remove(removed);
+        
+        assert removed == bottom;
+        bottom = queue.last();
+        //System.out.println("    now re-set bottom: " + bottom + " queue=" + queue);
+      } else if (queue.size() == topN) {
+        // Queue just filled up:
+        bottom = queue.last();
+        //System.out.println("    now set bottom: " + bottom);
+      }
+    }
+
+    public MinResult[] search() throws IOException {
+      //System.out.println("  search topN=" + topN);
+      final FST.Arc<Long> scratchArc = new FST.Arc<Long>();
+
+      final List<MinResult> results = new ArrayList<MinResult>();
+
+      final Long NO_OUTPUT = fst.outputs.getNoOutput();
+
+      // TODO: we could enable FST to sorting arcs by weight
+      // as it freezes... can easily do this on first pass
+      // (w/o requiring rewrite)
+
+      // TODO: maybe we should make an FST.INPUT_TYPE.BYTE0.5!?
+      // (nibbles)
+
+      // For each top N path:
+      while (results.size() < topN) {
+        //System.out.println("\nfind next path");
+
+        FSTPath path;
+
+        if (queue == null) {
+
+          if (results.size() != 0) {
+            // Ran out of paths
+            break;
+          }
+
+          // First pass (top path): start from original fromNode
+          if (topN > 1) {
+            queue = new TreeSet<FSTPath>();
+          }
+
+          long minArcCost = Long.MAX_VALUE;
+          FST.Arc<Long> minArc = null;
+
+          path = new FSTPath(0, fromNode);
+          fst.readFirstTargetArc(fromNode, path.arc);
+
+          // Bootstrap: find the min starting arc
+          while (true) {
+            long arcScore = path.arc.output;
+            //System.out.println("  arc=" + (char) path.arc.label + " cost=" + arcScore);
+            if (arcScore < minArcCost) {
+              minArcCost = arcScore;
+              minArc = scratchArc.copyFrom(path.arc);
+              //System.out.println("    **");
+            }
+            if (queue != null) {
+              addIfCompetitive(path);
+            }
+            if (path.arc.isLast()) {
+              break;
+            }
+            fst.readNextArc(path.arc);
+          }
+
+          assert minArc != null;
+
+          if (queue != null) {
+            // Remove top path since we are now going to
+            // pursue it:
+            // only on java6: see replacement 3 lines below
+            // path = queue.pollFirst();
+            assert !queue.isEmpty();
+            path = queue.first();
+            queue.remove(path);
+            
+            //System.out.println("  remove init path=" + path);
+            assert path.arc.label == minArc.label;
+            if (bottom != null && queue.size() == topN-1) {
+              bottom = queue.last();
+              //System.out.println("    set init bottom: " + bottom);
+            }
+          } else {
+            path.arc.copyFrom(minArc);
+            path.input.grow(1);
+            path.input.ints[0] = minArc.label;
+            path.input.length = 1;
+            path.cost = minArc.output;
+          }
+
+        } else {
+          // below is only on java5: see replacement code below
+          /*
+          path = queue.pollFirst();
+          if (path == null) {
+            // There were less than topN paths available:
+            break;
+          }
+          */
+          
+          // must do this empty check, first() throws exception.
+          if (queue.isEmpty()) {
+            // There were less than topN paths available:
+            path = null;
+            break;
+          } else {
+            path = queue.first();
+            queue.remove(path);
+          }
+        }
+
+        if (path.arc.label == FST.END_LABEL) {
+          //System.out.println("    empty string!  cost=" + path.cost);
+          // Empty string!
+          path.input.length--;
+          results.add(new MinResult(path.input, path.cost));
+          continue;
+        }
+
+        if (results.size() == topN-1) {
+          // Last path -- don't bother w/ queue anymore:
+          queue = null;
+        }
+
+        //System.out.println("  path: " + path);
+        
+        // We take path and find its "0 output completion",
+        // ie, just keep traversing the first arc with
+        // NO_OUTPUT that we can find, since this must lead
+        // to the minimum path that completes from
+        // path.arc.
+
+        // For each input letter:
+        while (true) {
+
+          //System.out.println("\n    cycle path: " + path);
+
+          fst.readFirstTargetArc(path.arc, path.arc);
+
+          // For each arc leaving this node:
+          boolean foundZero = false;
+          while(true) {
+            //System.out.println("      arc=" + (char) path.arc.label + " cost=" + path.arc.output);
+            if (path.arc.output == NO_OUTPUT) {
+              if (queue == null) {
+                foundZero = true;
+                break;
+              } else if (!foundZero) {
+                scratchArc.copyFrom(path.arc);
+                foundZero = true;
+              } else {
+                addIfCompetitive(path);
+              }
+            } else if (queue != null) {
+              addIfCompetitive(path);
+            }
+            if (path.arc.isLast()) {
+              break;
+            }
+            fst.readNextArc(path.arc);
+          }
+
+          assert foundZero;
+
+          if (queue != null) {
+            // TODO: maybe we can save this copyFrom if we
+            // are more clever above... eg on finding the
+            // first NO_OUTPUT arc we'd switch to using
+            // scratchArc
+            path.arc.copyFrom(scratchArc);
+          }
+
+          if (path.arc.label == FST.END_LABEL) {
+            // Add final output:
+            //System.out.println("    done!: " + path);
+            results.add(new MinResult(path.input, path.cost + path.arc.output));
+            break;
+          } else {
+            path.input.grow(1+path.input.length);
+            path.input.ints[path.input.length] = path.arc.label;
+            path.input.length++;
+            path.cost += path.arc.output;
+          }
+        }
+      }
+    
+      return results.toArray(new MinResult[results.size()]);
+    }
+  }
+
+  // TODO: parameterize the FST type <T> and allow passing in a
+  // comparator; eg maybe your output is a PairOutput and
+  // one of the outputs in the pair is monotonic so you
+  // compare by that
+
+  public final static class MinResult implements Comparable<MinResult> {
+    public final IntsRef input;
+    public final long output;
+    public MinResult(IntsRef input, long output) {
+      this.input = input;
+      this.output = output;
+    }
+
+    //@Override
+    public int compareTo(MinResult other) {
+      if (output < other.output) {
+        return -1;
+      } else if (output > other.output) {
+        return 1;
+      } else {
+        return input.compareTo(other.input);
+      }
+    }
+  }
+
+  /** Starting from node, find the top N min cost (Long
+   *  output) completions to a final node.
+   *
+   *  <p>NOTE: you must share the outputs when you build the
+   *  FST (pass doShare=true to {@link
+   *  PositiveIntOutputs#getSingleton}). */
+
+  public static MinResult[] shortestPaths(FST<Long> fst, FST.Arc<Long> fromNode, int topN) throws IOException {
+    return new TopNSearcher(fst, fromNode, topN).search();
+  } 
+
   /**
    * Dumps an {@link FST} to a GraphViz's <code>dot</code> language description
    * for visualization. Example of use:

Modified: lucene/dev/branches/branch_3x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_3x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Sun Feb 19 17:18:35 2012
@@ -51,6 +51,7 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST.BytesReader;
 
 public class TestFSTs extends LuceneTestCase {
 
@@ -1977,6 +1978,119 @@ public class TestFSTs extends LuceneTest
     assertEquals(42, arc.output.longValue());
   }
 
+  public void testShortestPaths() throws Exception {
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
+
+    final IntsRef scratch = new IntsRef();
+    builder.add(Util.toIntsRef(new BytesRef("aab"), scratch), 22L);
+    builder.add(Util.toIntsRef(new BytesRef("aac"), scratch), 7L);
+    builder.add(Util.toIntsRef(new BytesRef("ax"), scratch), 17L);
+    final FST<Long> fst = builder.finish();
+    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
+    //Util.toDot(fst, w, false, false);
+    //w.close();
+
+    Util.MinResult[] r = Util.shortestPaths(fst,
+                                           fst.getFirstArc(new FST.Arc<Long>()),
+                                           3);
+    assertEquals(3, r.length);
+
+    assertEquals(Util.toIntsRef(new BytesRef("aac"), scratch), r[0].input);
+    assertEquals(7, r[0].output);
+
+    assertEquals(Util.toIntsRef(new BytesRef("ax"), scratch), r[1].input);
+    assertEquals(17, r[1].output);
+
+    assertEquals(Util.toIntsRef(new BytesRef("aab"), scratch), r[2].input);
+    assertEquals(22, r[2].output);
+  }
+  
+  public void testShortestPathsRandom() throws Exception {
+    int numWords = atLeast(1000);
+    
+    final TreeMap<String,Long> slowCompletor = new TreeMap<String,Long>();
+    final TreeSet<String> allPrefixes = new TreeSet<String>();
+    
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
+    final IntsRef scratch = new IntsRef();
+    
+    for (int i = 0; i < numWords; i++) {
+      String s;
+      while (true) {
+        s = _TestUtil.randomSimpleString(random);
+        if (!slowCompletor.containsKey(s)) {
+          break;
+        }
+      }
+      
+      for (int j = 1; j < s.length(); j++) {
+        allPrefixes.add(s.substring(0, j));
+      }
+      int weight = _TestUtil.nextInt(random, 1, 100); // weights 1..100
+      slowCompletor.put(s, (long)weight);
+    }
+    
+    for (Map.Entry<String,Long> e : slowCompletor.entrySet()) {
+      //System.out.println("add: " + e);
+      builder.add(Util.toIntsRef(new BytesRef(e.getKey()), scratch), e.getValue());
+    }
+    
+    final FST<Long> fst = builder.finish();
+    //System.out.println("SAVE out.dot");
+    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
+    //Util.toDot(fst, w, false, false);
+    //w.close();
+    
+    BytesReader reader = fst.getBytesReader(0);
+    
+    //System.out.println("testing: " + allPrefixes.size() + " prefixes");
+    for (String prefix : allPrefixes) {
+      // 1. run prefix against fst, then complete by value
+      //System.out.println("TEST: " + prefix);
+    
+      long prefixOutput = 0;
+      FST.Arc<Long> arc = fst.getFirstArc(new FST.Arc<Long>());
+      for(int idx=0;idx<prefix.length();idx++) {
+        if (fst.findTargetArc((int) prefix.charAt(idx), arc, arc, reader) == null) {
+          fail();
+        }
+        prefixOutput += arc.output;
+      }
+
+      final int topN = _TestUtil.nextInt(random, 1, 10);
+
+      Util.MinResult[] r = Util.shortestPaths(fst, arc, topN);
+
+      // 2. go thru whole treemap (slowCompletor) and check its actually the best suggestion
+      final List<Util.MinResult> matches = new ArrayList<Util.MinResult>();
+
+      // TODO: could be faster... but its slowCompletor for a reason
+      for (Map.Entry<String,Long> e : slowCompletor.entrySet()) {
+        if (e.getKey().startsWith(prefix)) {
+          //System.out.println("  consider " + e.getKey());
+          matches.add(new Util.MinResult(Util.toIntsRef(new BytesRef(e.getKey().substring(prefix.length())), new IntsRef()),
+                                         e.getValue() - prefixOutput));
+        }
+      }
+
+      assertTrue(matches.size() > 0);
+      Collections.sort(matches);
+      if (matches.size() > topN) {
+        matches.subList(topN, matches.size()).clear();
+      }
+
+      assertEquals(matches.size(), r.length);
+
+      for(int hit=0;hit<r.length;hit++) {
+        //System.out.println("  check hit " + hit);
+        assertEquals(matches.get(hit).input, r[hit].input);
+        assertEquals(matches.get(hit).output, r[hit].output);
+      }
+    }
+  }
+
   public void testLargeOutputsOnArrayArcs() throws Exception {
     final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
     final Builder<BytesRef> builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, outputs);

Modified: lucene/dev/branches/branch_3x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/CHANGES.txt?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/solr/CHANGES.txt Sun Feb 19 17:18:35 2012
@@ -91,6 +91,9 @@ New Features
 * SOLR-3105: ElisionFilterFactory optionally allows the parameter 
   ignoreCase (default=false).  (Robert Muir)
 
+* LUCENE-3714: Add WFSTLookupFactory, a suggester that uses a weighted FST
+  for more fine-grained suggestions.  (Mike McCandless, Dawid Weiss, Robert Muir)
+
 Optimizations
 ----------------------
 * SOLR-1931: Speedup for LukeRequestHandler and admin/schema browser. New parameter

Modified: lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-spellchecker.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-spellchecker.xml?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-spellchecker.xml (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-spellchecker.xml Sun Feb 19 17:18:35 2012
@@ -137,6 +137,20 @@
       <bool name="exactMatchFirst">true</bool>
     </lst>
   </searchComponent>
+  
+  <!-- WFSTLookup suggest component -->
+  <searchComponent class="solr.SpellCheckComponent" name="suggest_wfst">
+    <lst name="spellchecker">
+      <str name="name">suggest_wfst</str>
+      <str name="classname">org.apache.solr.spelling.suggest.Suggester</str>
+      <str name="lookupImpl">org.apache.solr.spelling.suggest.fst.WFSTLookupFactory</str>
+      <str name="field">suggest</str>
+      <str name="storeDir">suggest_wfst</str>
+      <str name="buildOnCommit">true</str>
+      <!-- Suggester properties -->
+ 	  <bool name="exactMatchFirst">true</bool>
+    </lst>
+  </searchComponent>
 
   <!-- Suggest component -->
   <searchComponent class="solr.SpellCheckComponent" name="suggest">
@@ -192,6 +206,17 @@
     </arr>
   </requestHandler>
   
+  <!--  wfst (finite state automaton based) -->
+  <requestHandler class="org.apache.solr.handler.component.SearchHandler" name="/suggest_wfst">
+    <lst name="defaults">
+      <str name="spellcheck">true</str>
+      <str name="spellcheck.dictionary">suggest_wfst</str>
+      <str name="spellcheck.collate">false</str>
+    </lst>
+    <arr name="components">
+      <str>suggest_wfst</str>
+    </arr>
+  </requestHandler>
 
   <queryResponseWriter name="standard" class="solr.XMLResponseWriter"/>
   <queryResponseWriter name="useless" class="org.apache.solr.OutputWriterTest$UselessOutputWriter"/>

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterFSTTest.java Sun Feb 19 17:18:35 2012
@@ -1,5 +1,22 @@
 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.
+ */
+
 public class SuggesterFSTTest extends SuggesterTest {
   public SuggesterFSTTest() {
     super.requestUri = "/suggest_fst";

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java?rev=1291023&r1=1291022&r2=1291023&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/spelling/suggest/SuggesterTSTTest.java Sun Feb 19 17:18:35 2012
@@ -1,5 +1,22 @@
 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.
+ */
+
 public class SuggesterTSTTest extends SuggesterTest {
   public SuggesterTSTTest() {
     super.requestUri = "/suggest_tst";