You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/09/12 21:23:46 UTC

[2/2] lucene-solr:master: LUCENE-7439: clean up FuzzyQuery/FuzzyTermsEnum sources

LUCENE-7439: clean up FuzzyQuery/FuzzyTermsEnum sources


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/faf3bc31
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/faf3bc31
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/faf3bc31

Branch: refs/heads/master
Commit: faf3bc3134c6e5ba3e2caa15762524872e083152
Parents: 541a8fa
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Sep 12 17:23:24 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Sep 12 17:23:24 2016 -0400

----------------------------------------------------------------------
 .../org/apache/lucene/search/FuzzyQuery.java    |   2 +-
 .../apache/lucene/search/FuzzyTermsEnum.java    | 379 +++++++--------
 .../apache/lucene/search/TestFuzzyQuery.java    | 169 +++++++
 .../xml/builders/FuzzyLikeThisQueryBuilder.java |   4 +-
 .../sandbox/queries/FuzzyLikeThisQuery.java     | 312 ++++++------
 .../lucene/sandbox/queries/SlowFuzzyQuery.java  | 201 --------
 .../sandbox/queries/SlowFuzzyTermsEnum.java     | 263 ----------
 .../sandbox/queries/FuzzyLikeThisQueryTest.java |  14 +-
 .../sandbox/queries/TestSlowFuzzyQuery.java     | 487 -------------------
 .../lucene/search/spell/DirectSpellChecker.java |  20 +-
 10 files changed, 516 insertions(+), 1335 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
index 8e0cfff..3c1eacd 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
@@ -31,7 +31,7 @@ import org.apache.lucene.util.automaton.LevenshteinAutomata;
  * though you can explicitly choose classic Levenshtein by passing <code>false</code>
  * to the <code>transpositions</code> parameter.
  * 
- * <p>This query uses {@link MultiTermQuery.TopTermsScoringBooleanQueryRewrite}
+ * <p>This query uses {@link MultiTermQuery.TopTermsBlendedFreqScoringRewrite}
  * as default. So terms will be collected and scored according to their
  * edit distance. Only the top terms are used for building the {@link BooleanQuery}.
  * It is not recommended to change the rewrite mode for fuzzy queries.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
index e79dbf2..d30d34e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
@@ -17,12 +17,7 @@
 package org.apache.lucene.search;
 
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
@@ -35,10 +30,12 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
+import java.io.IOException;
+import java.util.Arrays;
+
 /** Subclass of TermsEnum for enumerating all terms that are similar
  * to the specified filter term.
  *
@@ -46,38 +43,46 @@ import org.apache.lucene.util.automaton.LevenshteinAutomata;
  * {@link BytesRef#compareTo}.  Each term in the enumeration is
  * greater than all that precede it.</p>
  */
-public class FuzzyTermsEnum extends TermsEnum {
+public final class FuzzyTermsEnum extends TermsEnum {
+
+  // NOTE: we can't subclass FilteredTermsEnum here because we need to sometimes change actualEnum:
   private TermsEnum actualEnum;
-  private BoostAttribute actualBoostAtt;
-  
-  private final BoostAttribute boostAtt =
-    attributes().addAttribute(BoostAttribute.class);
   
+  // We use this to communicate the score (boost) of the current matched term we are on back to
+  // MultiTermQuery.TopTermsBlendedFreqScoringRewrite that is collecting the best (default 50) matched terms:
+  private final BoostAttribute boostAtt;
+
+  // MultiTermQuery.TopTermsBlendedFreqScoringRewrite tells us the worst boost still in its queue using this att,
+  // which we use to know when we can reduce the automaton from ed=2 to ed=1, or ed=0 if only single top term is collected:
   private final MaxNonCompetitiveBoostAttribute maxBoostAtt;
+
+  // We use this to share the pre-built (once for the query) Levenshtein automata across segments:
   private final LevenshteinAutomataAttribute dfaAtt;
   
   private float bottom;
   private BytesRef bottomTerm;
-  
-  protected final float minSimilarity;
-  protected final float scale_factor;
-  
-  protected final int termLength;
-  
-  protected int maxEdits;
-  protected final boolean raw;
+  private final CompiledAutomaton automata[];
 
-  protected final Terms terms;
-  private final Term term;
-  protected final int termText[];
-  protected final int realPrefixLength;
-  
-  private final boolean transpositions;
+  private BytesRef queuedBottom;
+
+  final int termLength;
+
+  // Maximum number of edits we will accept.  This is either 2 or 1 (or, degenerately, 0) passed by the user originally,
+  // but as we collect terms, we can lower this (e.g. from 2 to 1) if we detect that the term queue is full, and all
+  // collected terms are ed=1:
+  private int maxEdits;
+
+  final Terms terms;
+  final Term term;
+  final int termText[];
+  final int realPrefixLength;
+
+  // True (the default, in FuzzyQuery) if a transposition should count as a single edit:
+  final boolean transpositions;
   
   /**
    * Constructor for enumeration of all terms from specified <code>reader</code> which share a prefix of
-   * length <code>prefixLength</code> with <code>term</code> and which have a fuzzy similarity &gt;
-   * <code>minSimilarity</code>.
+   * length <code>prefixLength</code> with <code>term</code> and which have at most {@code maxEdits} edits.
    * <p>
    * After calling the constructor the enumeration is already pointing to the first 
    * valid term if such a term exists. 
@@ -87,105 +92,88 @@ public class FuzzyTermsEnum extends TermsEnum {
    * thats contains information about competitive boosts during rewrite. It is also used
    * to cache DFAs between segment transitions.
    * @param term Pattern term.
-   * @param minSimilarity Minimum required similarity for terms from the reader. Pass an integer value
-   *        representing edit distance. Passing a fraction is deprecated.
+   * @param maxEdits Maximum edit distance.
    * @param prefixLength Length of required common prefix. Default value is 0.
    * @throws IOException if there is a low-level IO error
    */
   public FuzzyTermsEnum(Terms terms, AttributeSource atts, Term term, 
-      final float minSimilarity, final int prefixLength, boolean transpositions) throws IOException {
-    if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
-      throw new IllegalArgumentException("fractional edit distances are not allowed");
-    if (minSimilarity < 0.0f)
-      throw new IllegalArgumentException("minimumSimilarity cannot be less than 0");
-    if(prefixLength < 0)
+      final int maxEdits, final int prefixLength, boolean transpositions) throws IOException {
+    if (maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
+      throw new IllegalArgumentException("max edits must be 0.." + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + ", inclusive; got: " + maxEdits);
+    }
+    if (prefixLength < 0) {
       throw new IllegalArgumentException("prefixLength cannot be less than 0");
+    }
+    this.maxEdits = maxEdits;
     this.terms = terms;
     this.term = term;
-
+    
     // convert the string into a utf32 int[] representation for fast comparisons
     final String utf16 = term.text();
     this.termText = new int[utf16.codePointCount(0, utf16.length())];
-    for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp))
-           termText[j++] = cp = utf16.codePointAt(i);
+    for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp)) {
+      termText[j++] = cp = utf16.codePointAt(i);
+    }
     this.termLength = termText.length;
+
     this.dfaAtt = atts.addAttribute(LevenshteinAutomataAttribute.class);
+    this.maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
+
+    // NOTE: boostAtt must pulled from attributes() not from atts!  This is because TopTermsRewrite looks for boostAtt from this TermsEnum's
+    // private attributes() and not the global atts passed to us from MultiTermQuery:
+    this.boostAtt = attributes().addAttribute(BoostAttribute.class);
 
     //The prefix could be longer than the word.
     //It's kind of silly though.  It means we must match the entire word.
     this.realPrefixLength = prefixLength > termLength ? termLength : prefixLength;
-    // if minSimilarity >= 1, we treat it as number of edits
-    if (minSimilarity >= 1f) {
-      this.minSimilarity = 0; // just driven by number of edits
-      maxEdits = (int) minSimilarity;
-      raw = true;
-    } else {
-      this.minSimilarity = minSimilarity;
-      // calculate the maximum k edits for this similarity
-      maxEdits = initialMaxDistance(this.minSimilarity, termLength);
-      raw = false;
-    }
-    if (transpositions && maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
-      throw new UnsupportedOperationException("with transpositions enabled, distances > " 
-        + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + " are not supported ");
-    }
     this.transpositions = transpositions;
-    this.scale_factor = 1.0f / (1.0f - this.minSimilarity);
 
-    this.maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
+    CompiledAutomaton[] prevAutomata = dfaAtt.automata();
+    if (prevAutomata == null) {
+      prevAutomata = new CompiledAutomaton[maxEdits+1];
+
+      LevenshteinAutomata builder = 
+        new LevenshteinAutomata(UnicodeUtil.newString(termText, realPrefixLength, termText.length - realPrefixLength), transpositions);
+
+      String prefix = UnicodeUtil.newString(termText, 0, realPrefixLength);
+      for (int i = 0; i <= maxEdits; i++) {
+        Automaton a = builder.toAutomaton(i, prefix);
+        prevAutomata[i] = new CompiledAutomaton(a, true, false);
+      }
+
+      // first segment computes the automata, and we share with subsequent segments via this Attribute:
+      dfaAtt.setAutomata(prevAutomata);
+    }
+
+    this.automata = prevAutomata;
     bottom = maxBoostAtt.getMaxNonCompetitiveBoost();
     bottomTerm = maxBoostAtt.getCompetitiveTerm();
-    bottomChanged(null, true);
+    bottomChanged(null);
   }
   
   /**
    * return an automata-based enum for matching up to editDistance from
    * lastTerm, if possible
    */
-  protected TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm)
-      throws IOException {
-    final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
-    if (editDistance < runAutomata.size()) {
-      //System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
-      final CompiledAutomaton compiled = runAutomata.get(editDistance);
-      return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRefBuilder())),
-                                         runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
+  private TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm) throws IOException {
+    assert editDistance < automata.length;
+    final CompiledAutomaton compiled = automata[editDistance];
+    BytesRef initialSeekTerm;
+    if (lastTerm == null) {
+      // This is the first enum we are pulling:
+      initialSeekTerm = null;
     } else {
-      return null;
+      // We are pulling this enum (e.g., ed=1) after iterating for a while already (e.g., ed=2):
+      initialSeekTerm = compiled.floor(lastTerm, new BytesRefBuilder());
     }
+    return terms.intersect(compiled, initialSeekTerm);
   }
 
-  /** initialize levenshtein DFAs up to maxDistance, if possible */
-  private List<CompiledAutomaton> initAutomata(int maxDistance) {
-    final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
-    //System.out.println("cached automata size: " + runAutomata.size());
-    if (runAutomata.size() <= maxDistance &&
-        maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
-      LevenshteinAutomata builder = 
-        new LevenshteinAutomata(UnicodeUtil.newString(termText, realPrefixLength, termText.length - realPrefixLength), transpositions);
-
-      String prefix = UnicodeUtil.newString(termText, 0, realPrefixLength);
-      for (int i = runAutomata.size(); i <= maxDistance; i++) {
-        Automaton a = builder.toAutomaton(i, prefix);
-        //System.out.println("compute automaton n=" + i);
-        runAutomata.add(new CompiledAutomaton(a, true, false));
-      }
-    }
-    return runAutomata;
-  }
-
-  /** swap in a new actual enum to proxy to */
-  protected void setEnum(TermsEnum actualEnum) {
-    this.actualEnum = actualEnum;
-    this.actualBoostAtt = actualEnum.attributes().addAttribute(BoostAttribute.class);
-  }
-  
   /**
    * fired when the max non-competitive boost has changed. this is the hook to
-   * swap in a smarter actualEnum
+   * swap in a smarter actualEnum.
    */
-  private void bottomChanged(BytesRef lastTerm, boolean init)
-      throws IOException {
+  private void bottomChanged(BytesRef lastTerm) throws IOException {
     int oldMaxEdits = maxEdits;
     
     // true if the last term encountered is lexicographically equal or after the bottom term in the PQ
@@ -193,49 +181,73 @@ public class FuzzyTermsEnum extends TermsEnum {
 
     // as long as the max non-competitive boost is >= the max boost
     // for some edit distance, keep dropping the max edit distance.
-    while (maxEdits > 0 && (termAfter ? bottom >= calculateMaxBoost(maxEdits) : bottom > calculateMaxBoost(maxEdits)))
+    while (maxEdits > 0) {
+      float maxBoost = 1.0f - ((float) maxEdits / (float) termLength);
+      if (bottom < maxBoost || (bottom == maxBoost && termAfter == false)) {
+        break;
+      }
       maxEdits--;
-    
-    if (oldMaxEdits != maxEdits || init) { // the maximum n has changed
-      maxEditDistanceChanged(lastTerm, maxEdits, init);
-    }
-  }
-  
-  protected void maxEditDistanceChanged(BytesRef lastTerm, int maxEdits, boolean init)
-      throws IOException {
-    TermsEnum newEnum = getAutomatonEnum(maxEdits, lastTerm);
-    // instead of assert, we do a hard check in case someone uses our enum directly
-    // assert newEnum != null;
-    if (newEnum == null) {
-      assert maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE;
-      throw new IllegalArgumentException("maxEdits cannot be > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE");
     }
-    setEnum(newEnum);
-  }
 
-  // for some raw min similarity and input term length, the maximum # of edits
-  private int initialMaxDistance(float minimumSimilarity, int termLen) {
-    return (int) ((1D-minimumSimilarity) * termLen);
-  }
-  
-  // for some number of edits, the maximum possible scaled boost
-  private float calculateMaxBoost(int nEdits) {
-    final float similarity = 1.0f - ((float) nEdits / (float) (termLength));
-    return (similarity - minSimilarity) * scale_factor;
+    // TODO: this opto could be improved, e.g. if the worst term in the queue is zzzz with ed=2, then, really, on the next segment, we
+    // should only be looking for ed=1 terms up until zzzz, then ed=2.  Tricky :)
+    
+    if (oldMaxEdits != maxEdits || lastTerm == null) {
+      // This is a very powerful optimization: the maximum edit distance has changed.  This happens because we collect only the top scoring
+      // N (= 50, by default) terms, and if e.g. maxEdits=2, and the queue is now full of matching terms, and we notice that the worst entry
+      // in that queue is ed=1, then we can switch the automata here to ed=1 which is a big speedup.
+      actualEnum = getAutomatonEnum(maxEdits, lastTerm);
+    }
   }
-
-  private BytesRef queuedBottom = null;
   
   @Override
   public BytesRef next() throws IOException {
+
     if (queuedBottom != null) {
-      bottomChanged(queuedBottom, false);
+      bottomChanged(queuedBottom);
       queuedBottom = null;
     }
     
-    BytesRef term = actualEnum.next();
-    boostAtt.setBoost(actualBoostAtt.getBoost());
-    
+
+    BytesRef term;
+
+    // while loop because we skip short terms even if they are within the specified edit distance (see the NOTE in FuzzyQuery class javadocs)
+    while (true) {
+
+      term = actualEnum.next();
+      if (term == null) {
+        // end
+        break;
+      }
+
+      int ed = maxEdits;
+      
+      // we know the outer DFA always matches.
+      // now compute exact edit distance
+      while (ed > 0) {
+        if (matches(term, ed - 1)) {
+          ed--;
+        } else {
+          break;
+        }
+      }
+      
+      if (ed == 0) { // exact match
+        boostAtt.setBoost(1.0F);
+        break;
+      } else {
+        final int codePointCount = UnicodeUtil.codePointCount(term);
+        int minTermLength = Math.min(codePointCount, termLength);
+
+        // only accept a matching term if it's longer than the edit distance:
+        if (minTermLength > ed) {
+          float similarity = 1.0f - (float) ed / (float) minTermLength;
+          boostAtt.setBoost(similarity);
+          break;
+        }
+      }
+    }
+      
     final float bottom = maxBoostAtt.getMaxNonCompetitiveBoost();
     final BytesRef bottomTerm = maxBoostAtt.getCompetitiveTerm();
     if (term != null && (bottom != this.bottom || bottomTerm != this.bottomTerm)) {
@@ -243,11 +255,18 @@ public class FuzzyTermsEnum extends TermsEnum {
       this.bottomTerm = bottomTerm;
       // clone the term before potentially doing something with it
       // this is a rare but wonderful occurrence anyway
+
+      // We must delay bottomChanged until the next next() call otherwise we mess up docFreq(), etc., for the current term:
       queuedBottom = BytesRef.deepCopyOf(term);
     }
     
     return term;
   }
+
+  /** returns true if term is within k edits of the query term */
+  private boolean matches(BytesRef termIn, int k) {
+    return k == 0 ? termIn.equals(term.bytes()) : automata[k].runAutomaton.run(termIn.bytes, termIn.offset, termIn.length);
+  }
   
   // proxy all other enum calls to the actual enum
   @Override
@@ -301,108 +320,42 @@ public class FuzzyTermsEnum extends TermsEnum {
   }
 
   /**
-   * Implement fuzzy enumeration with Terms.intersect.
-   * <p>
-   * This is the fastest method as opposed to LinearFuzzyTermsEnum:
-   * as enumeration is logarithmic to the number of terms (instead of linear)
-   * and comparison is linear to length of the term (rather than quadratic)
-   */
-  private class AutomatonFuzzyTermsEnum extends FilteredTermsEnum {
-    private final ByteRunAutomaton matchers[];
-    
-    private final BytesRef termRef;
-    
-    private final BoostAttribute boostAtt =
-      attributes().addAttribute(BoostAttribute.class);
-    
-    public AutomatonFuzzyTermsEnum(TermsEnum tenum, CompiledAutomaton compiled[]) {
-      super(tenum, false);
-      this.matchers = new ByteRunAutomaton[compiled.length];
-      for (int i = 0; i < compiled.length; i++)
-        this.matchers[i] = compiled[i].runAutomaton;
-      termRef = new BytesRef(term.text());
-    }
-
-    /** finds the smallest Lev(n) DFA that accepts the term. */
-    @Override
-    protected AcceptStatus accept(BytesRef term) {    
-      //System.out.println("AFTE.accept term=" + term);
-      int ed = matchers.length - 1;
-      
-      // we are wrapping either an intersect() TermsEnum or an AutomatonTermsENum,
-      // so we know the outer DFA always matches.
-      // now compute exact edit distance
-      while (ed > 0) {
-        if (matches(term, ed - 1)) {
-          ed--;
-        } else {
-          break;
-        }
-      }
-      //System.out.println("CHECK term=" + term.utf8ToString() + " ed=" + ed);
-      
-      // scale to a boost and return (if similarity > minSimilarity)
-      if (ed == 0) { // exact match
-        boostAtt.setBoost(1.0F);
-        //System.out.println("  yes");
-        return AcceptStatus.YES;
-      } else {
-        final int codePointCount = UnicodeUtil.codePointCount(term);
-        final float similarity = 1.0f - ((float) ed / (float) 
-            (Math.min(codePointCount, termLength)));
-        if (similarity > minSimilarity) {
-          boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
-          //System.out.println("  yes");
-          return AcceptStatus.YES;
-        } else {
-          return AcceptStatus.NO;
-        }
-      }
-    }
-    
-    /** returns true if term is within k edits of the query term */
-    final boolean matches(BytesRef term, int k) {
-      return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
-    }
-  }
-
-  /** @lucene.internal */
-  public float getMinSimilarity() {
-    return minSimilarity;
-  }
-  
-  /** @lucene.internal */
-  public float getScaleFactor() {
-    return scale_factor;
-  }
-  
-  /**
    * reuses compiled automata across different segments,
    * because they are independent of the index
    * @lucene.internal */
   public static interface LevenshteinAutomataAttribute extends Attribute {
-    public List<CompiledAutomaton> automata();
+    public CompiledAutomaton[] automata();
+    public void setAutomata(CompiledAutomaton[] automata);
   }
     
   /** 
    * Stores compiled automata as a list (indexed by edit distance)
    * @lucene.internal */
   public static final class LevenshteinAutomataAttributeImpl extends AttributeImpl implements LevenshteinAutomataAttribute {
-    private final List<CompiledAutomaton> automata = new ArrayList<>();
+    private CompiledAutomaton[] automata;
       
     @Override
-    public List<CompiledAutomaton> automata() {
+    public CompiledAutomaton[] automata() {
       return automata;
     }
 
     @Override
+    public void setAutomata(CompiledAutomaton[] automata) {
+      this.automata = automata;
+    }
+
+    @Override
     public void clear() {
-      automata.clear();
+      automata = null;
     }
 
     @Override
     public int hashCode() {
-      return automata.hashCode();
+      if (automata == null) {
+        return 0;
+      } else {
+        return automata.hashCode();
+      }
     }
 
     @Override
@@ -411,15 +364,17 @@ public class FuzzyTermsEnum extends TermsEnum {
         return true;
       if (!(other instanceof LevenshteinAutomataAttributeImpl))
         return false;
-      return automata.equals(((LevenshteinAutomataAttributeImpl) other).automata);
+      return Arrays.equals(automata, ((LevenshteinAutomataAttributeImpl) other).automata);
     }
 
     @Override
-    public void copyTo(AttributeImpl target) {
-      final List<CompiledAutomaton> targetAutomata =
-        ((LevenshteinAutomataAttribute) target).automata();
-      targetAutomata.clear();
-      targetAutomata.addAll(automata);
+    public void copyTo(AttributeImpl _target) {
+      LevenshteinAutomataAttribute target = (LevenshteinAutomataAttribute) _target;
+      if (automata == null) {
+        target.setAutomata(null);
+      } else {
+        target.setAutomata(automata);
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
index 79dc157..a59449c 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestFuzzyQuery.java
@@ -19,12 +19,16 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.RandomIndexWriter;
@@ -32,7 +36,10 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
 /**
@@ -489,4 +496,166 @@ public class TestFuzzyQuery extends LuceneTestCase {
     doc.add(newTextField("field", text, Field.Store.YES));
     writer.addDocument(doc);
   }
+
+  private String randomSimpleString(int digits) {
+    int termLength = TestUtil.nextInt(random(), 1, 8);
+    char[] chars = new char[termLength];
+    for(int i=0;i<termLength;i++) {
+      chars[i] = (char) ('a' + random().nextInt(digits));
+    }
+    return new String(chars);
+  }
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  public void testRandom() throws Exception {
+    int numTerms = atLeast(100);
+    int digits = TestUtil.nextInt(random(), 2, 3);
+    Set<String> terms = new HashSet<>();
+    while (terms.size() < numTerms) {
+      terms.add(randomSimpleString(digits));
+    }
+
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    for(String term : terms) {
+      Document doc = new Document();
+      doc.add(new StringField("field", term, Field.Store.YES));
+      w.addDocument(doc);
+    }
+    DirectoryReader r = w.getReader();
+    IndexSearcher s = newSearcher(r);
+    int iters = atLeast(1000);
+    for(int iter=0;iter<iters;iter++) {
+      String queryTerm = randomSimpleString(digits);
+      int prefixLength = random().nextInt(queryTerm.length());
+      String queryPrefix = queryTerm.substring(0, prefixLength);
+
+      // we don't look at scores here:
+      Set<String>[] expected = new Set[3];
+      for(int ed=0;ed<3;ed++) {
+        expected[ed] = new HashSet<String>();
+      }
+      for(String term : terms) {
+        if (term.startsWith(queryPrefix) == false) {
+          continue;
+        }
+        int ed = getDistance(term, queryTerm);
+        if (Math.min(queryTerm.length(), term.length()) > ed) {
+          while (ed < 3) {
+            expected[ed].add(term);
+            ed++;
+          }
+        }
+      }
+
+      for(int ed=0;ed<3;ed++) {
+        FuzzyQuery query = new FuzzyQuery(new Term("field", queryTerm), ed, prefixLength, terms.size(), true);
+        TopDocs hits = s.search(query, terms.size());
+        Set<String> actual = new HashSet<>();
+        for(ScoreDoc hit : hits.scoreDocs) {
+          Document doc = s.doc(hit.doc);
+          actual.add(doc.get("field"));
+        }
+        if (actual.equals(expected[ed]) == false) {
+          StringBuilder sb = new StringBuilder();
+          sb.append("FAILED: query=" + queryTerm + " ed=" + ed + " prefixLength=" + prefixLength + "\n");
+
+          boolean first = true;
+          for(String term : actual) {
+            if (expected[ed].contains(term) == false) {
+              if (first) {
+                sb.append("  these matched but shouldn't:\n");
+                first = false;
+              }
+              sb.append("    " + term + "\n");
+            }
+          }
+          first = true;
+          for(String term : expected[ed]) {
+            if (actual.contains(term) == false) {
+              if (first) {
+                sb.append("  these did not match but should:\n");
+                first = false;
+              }
+              sb.append("    " + term + "\n");
+            }
+          }
+          throw new AssertionError(sb.toString());
+        }
+      }
+    }
+    
+    IOUtils.close(r, w, dir);
+  }
+
+  // Poached from LuceneLevenshteinDistance.java (from suggest module): it supports transpositions (treats them as ed=1, not ed=2)
+  private static int getDistance(String target, String other) {
+    IntsRef targetPoints;
+    IntsRef otherPoints;
+    int n;
+    int d[][]; // cost array
+    
+    // NOTE: if we cared, we could 3*m space instead of m*n space, similar to 
+    // what LevenshteinDistance does, except cycling thru a ring of three 
+    // horizontal cost arrays... but this comparator is never actually used by 
+    // DirectSpellChecker, it's only used for merging results from multiple shards 
+    // in "distributed spellcheck", and it's inefficient in other ways too...
+
+    // cheaper to do this up front once
+    targetPoints = toIntsRef(target);
+    otherPoints = toIntsRef(other);
+    n = targetPoints.length;
+    final int m = otherPoints.length;
+    d = new int[n+1][m+1];
+    
+    if (n == 0 || m == 0) {
+      if (n == m) {
+        return 0;
+      }
+      else {
+        return Math.max(n, m);
+      }
+    } 
+
+    // indexes into strings s and t
+    int i; // iterates through s
+    int j; // iterates through t
+
+    int t_j; // jth character of t
+
+    int cost; // cost
+
+    for (i = 0; i<=n; i++) {
+      d[i][0] = i;
+    }
+    
+    for (j = 0; j<=m; j++) {
+      d[0][j] = j;
+    }
+
+    for (j = 1; j<=m; j++) {
+      t_j = otherPoints.ints[j-1];
+
+      for (i=1; i<=n; i++) {
+        cost = targetPoints.ints[i-1]==t_j ? 0 : 1;
+        // minimum of cell to the left+1, to the top+1, diagonally left and up +cost
+        d[i][j] = Math.min(Math.min(d[i-1][j]+1, d[i][j-1]+1), d[i-1][j-1]+cost);
+        // transposition
+        if (i > 1 && j > 1 && targetPoints.ints[i-1] == otherPoints.ints[j-2] && targetPoints.ints[i-2] == otherPoints.ints[j-1]) {
+          d[i][j] = Math.min(d[i][j], d[i-2][j-2] + cost);
+        }
+      }
+    }
+    
+    return d[n][m];
+  }
+  
+  private static IntsRef toIntsRef(String s) {
+    IntsRef ref = new IntsRef(s.length()); // worst case
+    int utf16Len = s.length();
+    for (int i = 0, cp = 0; i < utf16Len; i += Character.charCount(cp)) {
+      cp = ref.ints[ref.length++] = Character.codePointAt(s, i);
+    }
+    return ref;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/FuzzyLikeThisQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/FuzzyLikeThisQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/FuzzyLikeThisQueryBuilder.java
index e7e9ad3..daf7354 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/FuzzyLikeThisQueryBuilder.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/FuzzyLikeThisQueryBuilder.java
@@ -21,8 +21,8 @@ import org.apache.lucene.queryparser.xml.DOMUtils;
 import org.apache.lucene.queryparser.xml.ParserException;
 import org.apache.lucene.queryparser.xml.QueryBuilder;
 import org.apache.lucene.sandbox.queries.FuzzyLikeThisQuery;
-import org.apache.lucene.sandbox.queries.SlowFuzzyQuery;
 import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.FuzzyQuery;
 import org.apache.lucene.search.Query;
 import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
@@ -33,7 +33,7 @@ import org.w3c.dom.NodeList;
 public class FuzzyLikeThisQueryBuilder implements QueryBuilder {
 
   private static final int DEFAULT_MAX_NUM_TERMS = 50;
-  private static final float DEFAULT_MIN_SIMILARITY = SlowFuzzyQuery.defaultMinSimilarity;
+  private static final float DEFAULT_MIN_SIMILARITY = FuzzyQuery.defaultMinSimilarity;
   private static final int DEFAULT_PREFIX_LENGTH = 1;
   private static final boolean DEFAULT_IGNORE_TF = false;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java
index 18ceba4..8bd7b89 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java
@@ -38,6 +38,7 @@ import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostAttribute;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.FuzzyTermsEnum;
 import org.apache.lucene.search.MaxNonCompetitiveBoostAttribute;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
@@ -46,6 +47,7 @@ import org.apache.lucene.search.similarities.TFIDFSimilarity;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
 /**
  * Fuzzifies ALL terms provided as strings and then picks the best n differentiating terms.
@@ -64,62 +66,62 @@ import org.apache.lucene.util.PriorityQueue;
  */
 public class FuzzyLikeThisQuery extends Query
 {
-    // TODO: generalize this query (at least it should not reuse this static sim!
-    // a better way might be to convert this into multitermquery rewrite methods.
-    // the rewrite method can 'average' the TermContext's term statistics (docfreq,totalTermFreq) 
-    // provided to TermQuery, so that the general idea is agnostic to any scoring system...
-    static TFIDFSimilarity sim=new ClassicSimilarity();
-    ArrayList<FieldVals> fieldVals=new ArrayList<>();
-    Analyzer analyzer;
+  // TODO: generalize this query (at least it should not reuse this static sim!
+  // a better way might be to convert this into multitermquery rewrite methods.
+  // the rewrite method can 'average' the TermContext's term statistics (docfreq,totalTermFreq) 
+  // provided to TermQuery, so that the general idea is agnostic to any scoring system...
+  static TFIDFSimilarity sim=new ClassicSimilarity();
+  ArrayList<FieldVals> fieldVals=new ArrayList<>();
+  Analyzer analyzer;
 
-    int MAX_VARIANTS_PER_TERM=50;
-    boolean ignoreTF=false;
-    private int maxNumTerms;
+  int MAX_VARIANTS_PER_TERM=50;
+  boolean ignoreTF=false;
+  private int maxNumTerms;
 
-    @Override
-    public int hashCode() {
-      int prime = 31;
-      int result = classHash();
-      result = prime * result + Objects.hashCode(analyzer);
-      result = prime * result + Objects.hashCode(fieldVals);
-      result = prime * result + (ignoreTF ? 1231 : 1237);
-      result = prime * result + maxNumTerms;
-      return result;
-    }
+  @Override
+  public int hashCode() {
+    int prime = 31;
+    int result = classHash();
+    result = prime * result + Objects.hashCode(analyzer);
+    result = prime * result + Objects.hashCode(fieldVals);
+    result = prime * result + (ignoreTF ? 1231 : 1237);
+    result = prime * result + maxNumTerms;
+    return result;
+  }
 
-    @Override
-    public boolean equals(Object other) {
-      return sameClassAs(other) &&
-             equalsTo(getClass().cast(other));
-    }
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+      equalsTo(getClass().cast(other));
+  }
 
-    private boolean equalsTo(FuzzyLikeThisQuery other) {
-      return Objects.equals(analyzer, other.analyzer) &&
-             Objects.equals(fieldVals, other.fieldVals) &&
-             ignoreTF == other.ignoreTF &&
-             maxNumTerms == other.maxNumTerms;
-    }
+  private boolean equalsTo(FuzzyLikeThisQuery other) {
+    return Objects.equals(analyzer, other.analyzer) &&
+      Objects.equals(fieldVals, other.fieldVals) &&
+      ignoreTF == other.ignoreTF &&
+      maxNumTerms == other.maxNumTerms;
+  }
 
-    /**
-     * 
-     * @param maxNumTerms The total number of terms clauses that will appear once rewritten as a BooleanQuery
-     */
-    public FuzzyLikeThisQuery(int maxNumTerms, Analyzer analyzer)
-    {
-        this.analyzer=analyzer;
-        this.maxNumTerms = maxNumTerms;
-    }
+  /**
+   * 
+   * @param maxNumTerms The total number of terms clauses that will appear once rewritten as a BooleanQuery
+   */
+  public FuzzyLikeThisQuery(int maxNumTerms, Analyzer analyzer)
+  {
+    this.analyzer=analyzer;
+    this.maxNumTerms = maxNumTerms;
+  }
 
-    class FieldVals
-    {
-      String queryString;
-      String fieldName;
-      float minSimilarity;
-      int prefixLength;
-    public FieldVals(String name, float similarity, int length, String queryString)
+  class FieldVals
+  {
+    String queryString;
+    String fieldName;
+    int maxEdits;
+    int prefixLength;
+    public FieldVals(String name, int maxEdits, int length, String queryString)
     {
       fieldName = name;
-      minSimilarity = similarity;
+      this.maxEdits = maxEdits;
       prefixLength = length;
       this.queryString = queryString;
     }
@@ -129,11 +131,11 @@ public class FuzzyLikeThisQuery extends Query
       final int prime = 31;
       int result = 1;
       result = prime * result
-          + ((fieldName == null) ? 0 : fieldName.hashCode());
-      result = prime * result + Float.floatToIntBits(minSimilarity);
+        + ((fieldName == null) ? 0 : fieldName.hashCode());
+      result = prime * result + maxEdits;
       result = prime * result + prefixLength;
       result = prime * result
-          + ((queryString == null) ? 0 : queryString.hashCode());
+        + ((queryString == null) ? 0 : queryString.hashCode());
       return result;
     }
 
@@ -151,9 +153,9 @@ public class FuzzyLikeThisQuery extends Query
           return false;
       } else if (!fieldName.equals(other.fieldName))
         return false;
-      if (Float.floatToIntBits(minSimilarity) != Float
-          .floatToIntBits(other.minSimilarity))
+      if (maxEdits != other.maxEdits) {
         return false;
+      }
       if (prefixLength != other.prefixLength)
         return false;
       if (queryString == null) {
@@ -166,18 +168,22 @@ public class FuzzyLikeThisQuery extends Query
     
 
 
-    }
+  }
     
-    /**
-     * Adds user input for "fuzzification" 
-     * @param queryString The string which will be parsed by the analyzer and for which fuzzy variants will be parsed
-     * @param minSimilarity The minimum similarity of the term variants (see FuzzyTermsEnum)
-     * @param prefixLength Length of required common prefix on variant terms (see FuzzyTermsEnum)
-     */
-    public void addTerms(String queryString, String fieldName,float minSimilarity, int prefixLength) 
-    {
-      fieldVals.add(new FieldVals(fieldName,minSimilarity,prefixLength,queryString));
+  /**
+   * Adds user input for "fuzzification" 
+   * @param queryString The string which will be parsed by the analyzer and for which fuzzy variants will be parsed
+   * @param minSimilarity The minimum similarity of the term variants; must be 0, 1 or 2 (see FuzzyTermsEnum)
+   * @param prefixLength Length of required common prefix on variant terms (see FuzzyTermsEnum)
+   */
+  public void addTerms(String queryString, String fieldName,float minSimilarity, int prefixLength) 
+  {
+    int maxEdits = (int) minSimilarity;
+    if (maxEdits != minSimilarity || maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
+      throw new IllegalArgumentException("minSimilarity must integer value between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + ", inclusive; got " + minSimilarity);
     }
+    fieldVals.add(new FieldVals(fieldName,maxEdits,prefixLength,queryString));
+  }
 
 
   private void addTerms(IndexReader reader, FieldVals f, ScoreTermQueue q) throws IOException {
@@ -202,7 +208,7 @@ public class FuzzyLikeThisQuery extends Query
           AttributeSource atts = new AttributeSource();
           MaxNonCompetitiveBoostAttribute maxBoostAtt =
             atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
-          SlowFuzzyTermsEnum fe = new SlowFuzzyTermsEnum(terms, atts, startTerm, f.minSimilarity, f.prefixLength);
+          FuzzyTermsEnum fe = new FuzzyTermsEnum(terms, atts, startTerm, f.maxEdits, f.prefixLength, true);
           //store the df so all variants use same idf
           int df = reader.docFreq(startTerm);
           int numVariants = 0;
@@ -225,9 +231,9 @@ public class FuzzyLikeThisQuery extends Query
           if (numVariants > 0) {
             int avgDf = totalVariantDocFreqs / numVariants;
             if (df == 0)//no direct match we can use as df for all variants
-            {
-              df = avgDf; //use avg df of all variants
-            }
+              {
+                df = avgDf; //use avg df of all variants
+              }
 
             // take the top variants (scored by edit distance) and reset the score
             // to include an IDF factor then add to the global queue for ranking
@@ -267,105 +273,105 @@ public class FuzzyLikeThisQuery extends Query
   }
 
   @Override
-    public Query rewrite(IndexReader reader) throws IOException
-    {
-        ScoreTermQueue q = new ScoreTermQueue(maxNumTerms);
-        //load up the list of possible terms
-        for (FieldVals f : fieldVals) {
-          addTerms(reader, f, q);
-        }
+  public Query rewrite(IndexReader reader) throws IOException
+  {
+    ScoreTermQueue q = new ScoreTermQueue(maxNumTerms);
+    //load up the list of possible terms
+    for (FieldVals f : fieldVals) {
+      addTerms(reader, f, q);
+    }
         
-        BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
         
-        //create BooleanQueries to hold the variants for each token/field pair and ensure it
-        // has no coord factor
-        //Step 1: sort the termqueries by term/field
-        HashMap<Term,ArrayList<ScoreTerm>> variantQueries=new HashMap<>();
-        int size = q.size();
-        for(int i = 0; i < size; i++)
-        {
-          ScoreTerm st = q.pop();
-          ArrayList<ScoreTerm> l= variantQueries.get(st.fuzziedSourceTerm);
-          if(l==null)
+    //create BooleanQueries to hold the variants for each token/field pair and ensure it
+    // has no coord factor
+    //Step 1: sort the termqueries by term/field
+    HashMap<Term,ArrayList<ScoreTerm>> variantQueries=new HashMap<>();
+    int size = q.size();
+    for(int i = 0; i < size; i++)
+      {
+        ScoreTerm st = q.pop();
+        ArrayList<ScoreTerm> l= variantQueries.get(st.fuzziedSourceTerm);
+        if(l==null)
           {
-              l=new ArrayList<>();
-              variantQueries.put(st.fuzziedSourceTerm,l);
+            l=new ArrayList<>();
+            variantQueries.put(st.fuzziedSourceTerm,l);
           }
-          l.add(st);
-        }
-        //Step 2: Organize the sorted termqueries into zero-coord scoring boolean queries
-        for (Iterator<ArrayList<ScoreTerm>> iter = variantQueries.values().iterator(); iter.hasNext();)
-        {
-            ArrayList<ScoreTerm> variants = iter.next();
-            if(variants.size()==1)
-            {
-                //optimize where only one selected variant
-                ScoreTerm st= variants.get(0);
+        l.add(st);
+      }
+    //Step 2: Organize the sorted termqueries into zero-coord scoring boolean queries
+    for (Iterator<ArrayList<ScoreTerm>> iter = variantQueries.values().iterator(); iter.hasNext();)
+      {
+        ArrayList<ScoreTerm> variants = iter.next();
+        if(variants.size()==1)
+          {
+            //optimize where only one selected variant
+            ScoreTerm st= variants.get(0);
+            Query tq = newTermQuery(reader, st.term);
+            // set the boost to a mix of IDF and score
+            bq.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD); 
+          }
+        else
+          {
+            BooleanQuery.Builder termVariants=new BooleanQuery.Builder();
+            for (Iterator<ScoreTerm> iterator2 = variants.iterator(); iterator2
+                   .hasNext();)
+              {
+                ScoreTerm st = iterator2.next();
+                // found a match
                 Query tq = newTermQuery(reader, st.term);
-                // set the boost to a mix of IDF and score
-                bq.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD); 
-            }
-            else
-            {
-                BooleanQuery.Builder termVariants=new BooleanQuery.Builder();
-                for (Iterator<ScoreTerm> iterator2 = variants.iterator(); iterator2
-                        .hasNext();)
-                {
-                    ScoreTerm st = iterator2.next();
-                    // found a match
-                    Query tq = newTermQuery(reader, st.term);
-                    // set the boost using the ScoreTerm's score
-                    termVariants.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD);          // add to query                    
-                }
-                bq.add(termVariants.build(), BooleanClause.Occur.SHOULD);          // add to query
-            }
-        }
-        //TODO possible alternative step 3 - organize above booleans into a new layer of field-based
-        // booleans with a minimum-should-match of NumFields-1?
-        return bq.build();
-    }
+                // set the boost using the ScoreTerm's score
+                termVariants.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD);          // add to query                    
+              }
+            bq.add(termVariants.build(), BooleanClause.Occur.SHOULD);          // add to query
+          }
+      }
+    //TODO possible alternative step 3 - organize above booleans into a new layer of field-based
+    // booleans with a minimum-should-match of NumFields-1?
+    return bq.build();
+  }
     
-    //Holds info for a fuzzy term variant - initially score is set to edit distance (for ranking best
-    // term variants) then is reset with IDF for use in ranking against all other
-    // terms/fields
-    private static class ScoreTerm{
-        public Term term;
-        public float score;
-        Term fuzziedSourceTerm;
+  //Holds info for a fuzzy term variant - initially score is set to edit distance (for ranking best
+  // term variants) then is reset with IDF for use in ranking against all other
+  // terms/fields
+  private static class ScoreTerm{
+    public Term term;
+    public float score;
+    Term fuzziedSourceTerm;
         
-        public ScoreTerm(Term term, float score, Term fuzziedSourceTerm){
-          this.term = term;
-          this.score = score;
-          this.fuzziedSourceTerm=fuzziedSourceTerm;
-        }
-      }
+    public ScoreTerm(Term term, float score, Term fuzziedSourceTerm){
+      this.term = term;
+      this.score = score;
+      this.fuzziedSourceTerm=fuzziedSourceTerm;
+    }
+  }
       
-      private static class ScoreTermQueue extends PriorityQueue<ScoreTerm> {        
-        public ScoreTermQueue(int size){
-          super(size);
-        }
-        
-        /* (non-Javadoc)
-         * @see org.apache.lucene.util.PriorityQueue#lessThan(java.lang.Object, java.lang.Object)
-         */
-        @Override
-        protected boolean lessThan(ScoreTerm termA, ScoreTerm termB) {
-          if (termA.score== termB.score)
-            return termA.term.compareTo(termB.term) > 0;
-          else
-            return termA.score < termB.score;
-        }
+  private static class ScoreTermQueue extends PriorityQueue<ScoreTerm> {        
+    public ScoreTermQueue(int size){
+      super(size);
+    }
         
-      }    
-      
     /* (non-Javadoc)
-     * @see org.apache.lucene.search.Query#toString(java.lang.String)
+     * @see org.apache.lucene.util.PriorityQueue#lessThan(java.lang.Object, java.lang.Object)
      */
     @Override
-    public String toString(String field)
-    {
-        return null;
+    protected boolean lessThan(ScoreTerm termA, ScoreTerm termB) {
+      if (termA.score== termB.score)
+        return termA.term.compareTo(termB.term) > 0;
+      else
+        return termA.score < termB.score;
     }
+        
+  }    
+      
+  /* (non-Javadoc)
+   * @see org.apache.lucene.search.Query#toString(java.lang.String)
+   */
+  @Override
+  public String toString(String field)
+  {
+    return null;
+  }
 
 
   public boolean isIgnoreTF()

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java
deleted file mode 100644
index fb4c202..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.sandbox.queries;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.SingleTermsEnum;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanQuery; // javadocs
-import org.apache.lucene.search.FuzzyQuery; // javadocs
-import org.apache.lucene.search.MultiTermQuery;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.automaton.LevenshteinAutomata;
-
-/** Implements the classic fuzzy search query. The similarity measurement
- * is based on the Levenshtein (edit distance) algorithm.
- * <p>
- * Note that, unlike {@link FuzzyQuery}, this query will silently allow
- * for a (possibly huge) number of edit distances in comparisons, and may
- * be extremely slow (comparing every term in the index).
- * 
- * @deprecated Use {@link FuzzyQuery} instead.
- */
-@Deprecated
-public class SlowFuzzyQuery extends MultiTermQuery {
-  
-  public final static float defaultMinSimilarity = LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE;
-  public final static int defaultPrefixLength = 0;
-  public final static int defaultMaxExpansions = 50;
-  
-  private float minimumSimilarity;
-  private int prefixLength;
-  private boolean termLongEnough = false;
-  
-  protected Term term;
-  
-  /**
-   * Create a new SlowFuzzyQuery that will match terms with a similarity 
-   * of at least <code>minimumSimilarity</code> to <code>term</code>.
-   * If a <code>prefixLength</code> &gt; 0 is specified, a common prefix
-   * of that length is also required.
-   * 
-   * @param term the term to search for
-   * @param minimumSimilarity a value between 0 and 1 to set the required similarity
-   *  between the query term and the matching terms. For example, for a
-   *  <code>minimumSimilarity</code> of <code>0.5</code> a term of the same length
-   *  as the query term is considered similar to the query term if the edit distance
-   *  between both terms is less than <code>length(term)*0.5</code>
-   *  <p>
-   *  Alternatively, if <code>minimumSimilarity</code> is &gt;= 1f, it is interpreted 
-   *  as a pure Levenshtein edit distance. For example, a value of <code>2f</code>
-   *  will match all terms within an edit distance of <code>2</code> from the 
-   *  query term. Edit distances specified in this way may not be fractional.
-   *  
-   * @param prefixLength length of common (non-fuzzy) prefix
-   * @param maxExpansions the maximum number of terms to match. If this number is
-   *  greater than {@link BooleanQuery#getMaxClauseCount} when the query is rewritten, 
-   *  then the maxClauseCount will be used instead.
-   * @throws IllegalArgumentException if minimumSimilarity is &gt;= 1 or &lt; 0
-   * or if prefixLength &lt; 0
-   */
-  public SlowFuzzyQuery(Term term, float minimumSimilarity, int prefixLength,
-      int maxExpansions) {
-    super(term.field());
-    this.term = term;
-    
-    if (minimumSimilarity >= 1.0f && minimumSimilarity != (int)minimumSimilarity)
-      throw new IllegalArgumentException("fractional edit distances are not allowed");
-    if (minimumSimilarity < 0.0f)
-      throw new IllegalArgumentException("minimumSimilarity < 0");
-    if (prefixLength < 0)
-      throw new IllegalArgumentException("prefixLength < 0");
-    if (maxExpansions < 0)
-      throw new IllegalArgumentException("maxExpansions < 0");
-    
-    setRewriteMethod(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(maxExpansions));
-    
-    String text = term.text();
-    int len = text.codePointCount(0, text.length());
-    if (len > 0 && (minimumSimilarity >= 1f || len > 1.0f / (1.0f - minimumSimilarity))) {
-      this.termLongEnough = true;
-    }
-    
-    this.minimumSimilarity = minimumSimilarity;
-    this.prefixLength = prefixLength;
-  }
-  
-  /**
-   * Calls {@link #SlowFuzzyQuery(Term, float) SlowFuzzyQuery(term, minimumSimilarity, prefixLength, defaultMaxExpansions)}.
-   */
-  public SlowFuzzyQuery(Term term, float minimumSimilarity, int prefixLength) {
-    this(term, minimumSimilarity, prefixLength, defaultMaxExpansions);
-  }
-  
-  /**
-   * Calls {@link #SlowFuzzyQuery(Term, float) SlowFuzzyQuery(term, minimumSimilarity, 0, defaultMaxExpansions)}.
-   */
-  public SlowFuzzyQuery(Term term, float minimumSimilarity) {
-    this(term, minimumSimilarity, defaultPrefixLength, defaultMaxExpansions);
-  }
-
-  /**
-   * Calls {@link #SlowFuzzyQuery(Term, float) SlowFuzzyQuery(term, defaultMinSimilarity, 0, defaultMaxExpansions)}.
-   */
-  public SlowFuzzyQuery(Term term) {
-    this(term, defaultMinSimilarity, defaultPrefixLength, defaultMaxExpansions);
-  }
-  
-  /**
-   * Returns the minimum similarity that is required for this query to match.
-   * @return float value between 0.0 and 1.0
-   */
-  public float getMinSimilarity() {
-    return minimumSimilarity;
-  }
-    
-  /**
-   * Returns the non-fuzzy prefix length. This is the number of characters at the start
-   * of a term that must be identical (not fuzzy) to the query term if the query
-   * is to match that term. 
-   */
-  public int getPrefixLength() {
-    return prefixLength;
-  }
-
-  @Override
-  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    if (!termLongEnough) {  // can only match if it's exact
-      return new SingleTermsEnum(terms.iterator(), term.bytes());
-    }
-    return new SlowFuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength);
-  }
-  
-  /**
-   * Returns the pattern term.
-   */
-  public Term getTerm() {
-    return term;
-  }
-    
-  @Override
-  public String toString(String field) {
-    final StringBuilder buffer = new StringBuilder();
-    if (!term.field().equals(field)) {
-        buffer.append(term.field());
-        buffer.append(":");
-    }
-    buffer.append(term.text());
-    buffer.append('~');
-    buffer.append(Float.toString(minimumSimilarity));
-    return buffer.toString();
-  }
-  
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + Float.floatToIntBits(minimumSimilarity);
-    result = prime * result + prefixLength;
-    result = prime * result + ((term == null) ? 0 : term.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (!super.equals(obj))
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    SlowFuzzyQuery other = (SlowFuzzyQuery) obj;
-    if (Float.floatToIntBits(minimumSimilarity) != Float
-        .floatToIntBits(other.minimumSimilarity))
-      return false;
-    if (prefixLength != other.prefixLength)
-      return false;
-    if (term == null) {
-      if (other.term != null)
-        return false;
-    } else if (!term.equals(other.term))
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java
deleted file mode 100644
index 8f466cc..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.sandbox.queries;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.FilteredTermsEnum;
-import org.apache.lucene.search.BoostAttribute;
-import org.apache.lucene.search.FuzzyTermsEnum;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRefBuilder;
-import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.UnicodeUtil;
-
-/** Potentially slow fuzzy TermsEnum for enumerating all terms that are similar
- * to the specified filter term.
- * <p> If the minSimilarity or maxEdits is greater than the Automaton's
- * allowable range, this backs off to the classic (brute force)
- * fuzzy terms enum method by calling FuzzyTermsEnum's getAutomatonEnum.
- * </p>
- * <p>Term enumerations are always ordered by
- * {@link BytesRef#compareTo}.  Each term in the enumeration is
- * greater than all that precede it.</p>
- * 
- * @deprecated Use {@link FuzzyTermsEnum} instead.
- */
-@Deprecated
-public final class SlowFuzzyTermsEnum extends FuzzyTermsEnum {
- 
-  public SlowFuzzyTermsEnum(Terms terms, AttributeSource atts, Term term,
-      float minSimilarity, int prefixLength) throws IOException {
-    super(terms, atts, term, minSimilarity, prefixLength, false);
-  }
-  
-  @Override
-  protected void maxEditDistanceChanged(BytesRef lastTerm, int maxEdits, boolean init)
-      throws IOException {
-    TermsEnum newEnum = getAutomatonEnum(maxEdits, lastTerm);
-    if (newEnum != null) {
-      setEnum(newEnum);
-    } else if (init) {
-      setEnum(new LinearFuzzyTermsEnum());      
-    }
-  }
-
-  /**
-   * Implement fuzzy enumeration with linear brute force.
-   */
-  private class LinearFuzzyTermsEnum extends FilteredTermsEnum {
-    /* Allows us save time required to create a new array
-     * every time similarity is called.
-     */
-    private int[] d;
-    private int[] p;
-    
-    // this is the text, minus the prefix
-    private final int[] text;
-    
-    private final BoostAttribute boostAtt =
-      attributes().addAttribute(BoostAttribute.class);
-    
-    /**
-     * Constructor for enumeration of all terms from specified <code>reader</code> which share a prefix of
-     * length <code>prefixLength</code> with <code>term</code> and which have a fuzzy similarity &gt;
-     * <code>minSimilarity</code>.
-     * <p>
-     * After calling the constructor the enumeration is already pointing to the first 
-     * valid term if such a term exists.
-     *
-     * @throws IOException If there is a low-level I/O error.
-     */
-    public LinearFuzzyTermsEnum() throws IOException {
-      super(terms.iterator());
-
-      this.text = new int[termLength - realPrefixLength];
-      System.arraycopy(termText, realPrefixLength, text, 0, text.length);
-      final String prefix = UnicodeUtil.newString(termText, 0, realPrefixLength);
-      prefixBytesRef = new BytesRef(prefix);
-      this.d = new int[this.text.length + 1];
-      this.p = new int[this.text.length + 1];
-      
-      setInitialSeekTerm(prefixBytesRef);
-    }
-    
-    private final BytesRef prefixBytesRef;
-    // used for unicode conversion from BytesRef byte[] to int[]
-    private final IntsRefBuilder utf32 = new IntsRefBuilder();
-    
-    /**
-     * <p>The termCompare method in FuzzyTermEnum uses Levenshtein distance to 
-     * calculate the distance between the given term and the comparing term. 
-     * </p>
-     * <p>If the minSimilarity is &gt;= 1.0, this uses the maxEdits as the comparison.
-     * Otherwise, this method uses the following logic to calculate similarity.
-     * <pre>
-     *   similarity = 1 - ((float)distance / (float) (prefixLength + Math.min(textlen, targetlen)));
-     *   </pre>
-     * where distance is the Levenshtein distance for the two words.
-     * </p>
-     * 
-     */
-    @Override
-    protected final AcceptStatus accept(BytesRef term) {
-      if (StringHelper.startsWith(term, prefixBytesRef)) {
-        utf32.copyUTF8Bytes(term);
-        final int distance = calcDistance(utf32.ints(), realPrefixLength, utf32.length() - realPrefixLength);
-       
-        //Integer.MIN_VALUE is the sentinel that Levenshtein stopped early
-        if (distance == Integer.MIN_VALUE){
-           return AcceptStatus.NO;
-        }
-        //no need to calc similarity, if raw is true and distance > maxEdits
-        if (raw == true && distance > maxEdits){
-              return AcceptStatus.NO;
-        } 
-        final float similarity = calcSimilarity(distance, (utf32.length() - realPrefixLength), text.length);
-        
-        //if raw is true, then distance must also be <= maxEdits by now
-        //given the previous if statement
-        if (raw == true ||
-              (raw == false && similarity > minSimilarity)) {
-          boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
-          return AcceptStatus.YES;
-        } else {
-           return AcceptStatus.NO;
-        }
-      } else {
-        return AcceptStatus.END;
-      }
-    }
-    
-    /******************************
-     * Compute Levenshtein distance
-     ******************************/
-    
-    /**
-     * <p>calcDistance returns the Levenshtein distance between the query term
-     * and the target term.</p>
-     * 
-     * <p>Embedded within this algorithm is a fail-fast Levenshtein distance
-     * algorithm.  The fail-fast algorithm differs from the standard Levenshtein
-     * distance algorithm in that it is aborted if it is discovered that the
-     * minimum distance between the words is greater than some threshold.
-
-     * <p>Levenshtein distance (also known as edit distance) is a measure of similarity
-     * between two strings where the distance is measured as the number of character
-     * deletions, insertions or substitutions required to transform one string to
-     * the other string.
-     * @param target the target word or phrase
-     * @param offset the offset at which to start the comparison
-     * @param length the length of what's left of the string to compare
-     * @return the number of edits or Integer.MIN_VALUE if the edit distance is
-     * greater than maxDistance.
-     */
-    private final int calcDistance(final int[] target, int offset, int length) {
-      final int m = length;
-      final int n = text.length;
-      if (n == 0)  {
-        //we don't have anything to compare.  That means if we just add
-        //the letters for m we get the new word
-        return m;
-      }
-      if (m == 0) {
-        return n;
-      }
-      
-      final int maxDistance = calculateMaxDistance(m);
-      
-      if (maxDistance < Math.abs(m-n)) {
-        //just adding the characters of m to n or vice-versa results in
-        //too many edits
-        //for example "pre" length is 3 and "prefixes" length is 8.  We can see that
-        //given this optimal circumstance, the edit distance cannot be less than 5.
-        //which is 8-3 or more precisely Math.abs(3-8).
-        //if our maximum edit distance is 4, then we can discard this word
-        //without looking at it.
-        return Integer.MIN_VALUE;
-      }
-      
-      // init matrix d
-      for (int i = 0; i <=n; ++i) {
-        p[i] = i;
-      }
-      
-      // start computing edit distance
-      for (int j = 1; j<=m; ++j) { // iterates through target
-        int bestPossibleEditDistance = m;
-        final int t_j = target[offset+j-1]; // jth character of t
-        d[0] = j;
-
-        for (int i=1; i<=n; ++i) { // iterates through text
-          // minimum of cell to the left+1, to the top+1, diagonally left and up +(0|1)
-          if (t_j != text[i-1]) {
-            d[i] = Math.min(Math.min(d[i-1], p[i]),  p[i-1]) + 1;
-          } else {
-            d[i] = Math.min(Math.min(d[i-1]+1, p[i]+1),  p[i-1]);
-          }
-          bestPossibleEditDistance = Math.min(bestPossibleEditDistance, d[i]);
-        }
-
-        //After calculating row i, the best possible edit distance
-        //can be found by found by finding the smallest value in a given column.
-        //If the bestPossibleEditDistance is greater than the max distance, abort.
-
-        if (j > maxDistance && bestPossibleEditDistance > maxDistance) {  //equal is okay, but not greater
-          //the closest the target can be to the text is just too far away.
-          //this target is leaving the party early.
-          return Integer.MIN_VALUE;
-        }
-
-        // copy current distance counts to 'previous row' distance counts: swap p and d
-        int _d[] = p;
-        p = d;
-        d = _d;
-      }
-      
-      // our last action in the above loop was to switch d and p, so p now
-      // actually has the most recent cost counts
-
-      return p[n];
-    }
-    
-    private float calcSimilarity(int edits, int m, int n){
-      // this will return less than 0.0 when the edit distance is
-      // greater than the number of characters in the shorter word.
-      // but this was the formula that was previously used in FuzzyTermEnum,
-      // so it has not been changed (even though minimumSimilarity must be
-      // greater than 0.0)
-      
-      return 1.0f - ((float)edits / (float) (realPrefixLength + Math.min(n, m)));
-    }
-    
-    /**
-     * The max Distance is the maximum Levenshtein distance for the text
-     * compared to some other value that results in score that is
-     * better than the minimum similarity.
-     * @param m the length of the "other value"
-     * @return the maximum levenshtein distance that we care about
-     */
-    private int calculateMaxDistance(int m) {
-      return raw ? maxEdits : Math.min(maxEdits, 
-          (int)((1-minSimilarity) * (Math.min(text.length, m) + realPrefixLength)));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/faf3bc31/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/FuzzyLikeThisQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/FuzzyLikeThisQueryTest.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/FuzzyLikeThisQueryTest.java
index a8c8b51..e744c72c 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/FuzzyLikeThisQueryTest.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/FuzzyLikeThisQueryTest.java
@@ -77,7 +77,7 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
   //Tests that idf ranking is not favouring rare mis-spellings over a strong edit-distance match
   public void testClosestEditDistanceMatchComesFirst() throws Throwable {
     FuzzyLikeThisQuery flt = new FuzzyLikeThisQuery(10, analyzer);
-    flt.addTerms("smith", "name", 0.3f, 1);
+    flt.addTerms("smith", "name", 2, 1);
     Query q = flt.rewrite(searcher.getIndexReader());
     HashSet<Term> queryTerms = new HashSet<>();
     searcher.createWeight(q, true, 1f).extractTerms(queryTerms);
@@ -94,7 +94,7 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
   //Test multiple input words are having variants produced
   public void testMultiWord() throws Throwable {
     FuzzyLikeThisQuery flt = new FuzzyLikeThisQuery(10, analyzer);
-    flt.addTerms("jonathin smoth", "name", 0.3f, 1);
+    flt.addTerms("jonathin smoth", "name", 2, 1);
     Query q = flt.rewrite(searcher.getIndexReader());
     HashSet<Term> queryTerms = new HashSet<>();
     searcher.createWeight(q, true, 1f).extractTerms(queryTerms);
@@ -110,8 +110,8 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
   // LUCENE-4809
   public void testNonExistingField() throws Throwable {
     FuzzyLikeThisQuery flt = new FuzzyLikeThisQuery(10, analyzer);
-    flt.addTerms("jonathin smoth", "name", 0.3f, 1);
-    flt.addTerms("jonathin smoth", "this field does not exist", 0.3f, 1);
+    flt.addTerms("jonathin smoth", "name", 2, 1);
+    flt.addTerms("jonathin smoth", "this field does not exist", 2, 1);
     // don't fail here just because the field doesn't exits
     Query q = flt.rewrite(searcher.getIndexReader());
     HashSet<Term> queryTerms = new HashSet<>();
@@ -129,7 +129,7 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
   //Test bug found when first query word does not match anything
   public void testNoMatchFirstWordBug() throws Throwable {
     FuzzyLikeThisQuery flt = new FuzzyLikeThisQuery(10, analyzer);
-    flt.addTerms("fernando smith", "name", 0.3f, 1);
+    flt.addTerms("fernando smith", "name", 2, 1);
     Query q = flt.rewrite(searcher.getIndexReader());
     HashSet<Term> queryTerms = new HashSet<>();
     searcher.createWeight(q, true, 1f).extractTerms(queryTerms);
@@ -144,9 +144,9 @@ public class FuzzyLikeThisQueryTest extends LuceneTestCase {
   public void testFuzzyLikeThisQueryEquals() {
     Analyzer analyzer = new MockAnalyzer(random());
     FuzzyLikeThisQuery fltq1 = new FuzzyLikeThisQuery(10, analyzer);
-    fltq1.addTerms("javi", "subject", 0.5f, 2);
+    fltq1.addTerms("javi", "subject", 2, 2);
     FuzzyLikeThisQuery fltq2 = new FuzzyLikeThisQuery(10, analyzer);
-    fltq2.addTerms("javi", "subject", 0.5f, 2);
+    fltq2.addTerms("javi", "subject", 2, 2);
     assertEquals("FuzzyLikeThisQuery with same attributes is not equal", fltq1,
         fltq2);
   }