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 2013/07/18 16:36:34 UTC

svn commit: r1504490 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/analysis/ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/ solr/core/src/...

Author: mikemccand
Date: Thu Jul 18 14:36:34 2013
New Revision: 1504490

URL: http://svn.apache.org/r1504490
Log:
LUCENE-5030: FuzzySuggester can optionally measure edits in Unicode code points instead of UTF8 bytes

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Jul 18 14:36:34 2013
@@ -60,6 +60,10 @@ New features
 * LUCENE-5098: New broadword utility methods in oal.util.BroadWord.
   (Paul Elschot via Adrien Grand, Dawid Weiss)
 
+* LUCENE-5030: FuzzySuggester now supports optional unicodeAware
+  (default is false).  If true then edits are measured in Unicode code
+  points instead of UTF8 bytes.  (Artem Lukanin via Mike McCandless)
+
 API Changes
 
 * LUCENE-5094: Add ramBytesUsed() to MultiDocValues.OrdinalMap.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java Thu Jul 18 14:36:34 2013
@@ -32,7 +32,8 @@ import org.apache.lucene.util.automaton.
 // TODO: maybe also toFST?  then we can translate atts into FST outputs/weights
 
 /** Consumes a TokenStream and creates an {@link Automaton}
- *  where the transition labels are UTF8 bytes from the {@link
+ *  where the transition labels are UTF8 bytes (or Unicode 
+ *  code points if unicodeArcs is true) from the {@link
  *  TermToBytesRefAttribute}.  Between tokens we insert
  *  POS_SEP and for holes we insert HOLE.
  *
@@ -40,6 +41,7 @@ import org.apache.lucene.util.automaton.
 public class TokenStreamToAutomaton {
 
   private boolean preservePositionIncrements;
+  private boolean unicodeArcs;
 
   /** Sole constructor. */
   public TokenStreamToAutomaton() {
@@ -51,6 +53,12 @@ public class TokenStreamToAutomaton {
     this.preservePositionIncrements = enablePositionIncrements;
   }
 
+  /** Whether to make transition labels Unicode code points instead of UTF8 bytes, 
+   *  <code>false</code> by default */
+  public void setUnicodeArcs(boolean unicodeArcs) {
+    this.unicodeArcs = unicodeArcs;
+  }
+
   private static class Position implements RollingBuffer.Resettable {
     // Any tokens that ended at our position arrive to this state:
     State arriving;
@@ -80,15 +88,16 @@ public class TokenStreamToAutomaton {
   }
 
   /** We create transition between two adjacent tokens. */
-  public static final int POS_SEP = 256;
+  public static final int POS_SEP = 0x001f;
 
   /** We add this arc to represent a hole. */
-  public static final int HOLE = 257;
+  public static final int HOLE = 0x001e;
 
   /** Pulls the graph (including {@link
    *  PositionLengthAttribute}) from the provided {@link
    *  TokenStream}, and creates the corresponding
-   *  automaton where arcs are bytes from each term. */
+   *  automaton where arcs are bytes (or Unicode code points 
+   *  if unicodeArcs = true) from each term. */
   public Automaton toAutomaton(TokenStream in) throws IOException {
     final Automaton a = new Automaton();
     boolean deterministic = true;
@@ -156,16 +165,34 @@ public class TokenStreamToAutomaton {
       final int endPos = pos + posLengthAtt.getPositionLength();
 
       termBytesAtt.fillBytesRef();
-      final BytesRef term2 = changeToken(term);
+      final BytesRef termUTF8 = changeToken(term);
+      int[] termUnicode = null;
       final Position endPosData = positions.get(endPos);
       if (endPosData.arriving == null) {
         endPosData.arriving = new State();
       }
 
       State state = posData.leaving;
-      for(int byteIDX=0;byteIDX<term2.length;byteIDX++) {
-        final State nextState = byteIDX == term2.length-1 ? endPosData.arriving : new State();
-        state.addTransition(new Transition(term2.bytes[term2.offset + byteIDX] & 0xff, nextState));
+      int termLen;
+      if (unicodeArcs) {
+        final String utf16 = termUTF8.utf8ToString();
+        termUnicode = new int[utf16.codePointCount(0, utf16.length())];
+        termLen = termUnicode.length;
+        for (int cp, i = 0, j = 0; i < utf16.length(); i += Character.charCount(cp))
+          termUnicode[j++] = cp = utf16.codePointAt(i);
+      } else {
+        termLen = termUTF8.length;
+      }
+
+      for(int byteIDX=0;byteIDX<termLen;byteIDX++) {
+        final State nextState = byteIDX == termLen-1 ? endPosData.arriving : new State();
+        int c;
+        if (unicodeArcs) {
+          c = termUnicode[byteIDX];
+        } else {
+          c = termUTF8.bytes[termUTF8.offset + byteIDX] & 0xff;
+        }
+        state.addTransition(new Transition(c, nextState));
         state = nextState;
       }
 

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Thu Jul 18 14:36:34 2013
@@ -159,7 +159,7 @@ public class AnalyzingSuggester extends 
 
   /** Represents the separation between tokens, if
    *  PRESERVE_SEP was specified */
-  private static final int SEP_LABEL = 0xff;
+  private static final int SEP_LABEL = '\u001F';
 
   /** Marks end of the analyzed input and start of dedup
    *  byte. */
@@ -306,44 +306,14 @@ public class AnalyzingSuggester extends 
     }
   }
 
-  /** Just escapes the 0xff byte (which we still for SEP). */
-  private static final class  EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
-
-    final BytesRef spare = new BytesRef();
-
-    @Override
-    protected BytesRef changeToken(BytesRef in) {
-      int upto = 0;
-      for(int i=0;i<in.length;i++) {
-        byte b = in.bytes[in.offset+i];
-        if (b == (byte) 0xff) {
-          if (spare.bytes.length == upto) {
-            spare.grow(upto+2);
-          }
-          spare.bytes[upto++] = (byte) 0xff;
-          spare.bytes[upto++] = b;
-        } else {
-          if (spare.bytes.length == upto) {
-            spare.grow(upto+1);
-          }
-          spare.bytes[upto++] = b;
-        }
-      }
-      spare.offset = 0;
-      spare.length = upto;
-      return spare;
-    }
+  /** Used by subclass to change the lookup automaton, if
+   *  necessary. */
+  protected Automaton convertAutomaton(Automaton a) {
+    return a;
   }
-
+  
   TokenStreamToAutomaton getTokenStreamToAutomaton() {
-    final TokenStreamToAutomaton tsta;
-    if (preserveSep) {
-      tsta = new EscapingTokenStreamToAutomaton();
-    } else {
-      // When we're not preserving sep, we don't steal 0xff
-      // byte, so we don't need to do any escaping:
-      tsta = new TokenStreamToAutomaton();
-    }
+    final TokenStreamToAutomaton tsta = new TokenStreamToAutomaton();
     tsta.setPreservePositionIncrements(preservePositionIncrements);
     return tsta;
   }
@@ -412,7 +382,7 @@ public class AnalyzingSuggester extends 
 
       return 0;
     }
-  };
+  }
 
   @Override
   public void build(TermFreqIterator iterator) throws IOException {
@@ -699,6 +669,14 @@ public class AnalyzingSuggester extends 
     }
 
     //System.out.println("lookup key=" + key + " num=" + num);
+    for (int i = 0; i < key.length(); i++) {
+      if (key.charAt(i) == 0x1E) {
+        throw new IllegalArgumentException("lookup key cannot contain HOLE character U+001E; this character is reserved");
+      }
+      if (key.charAt(i) == 0x1F) {
+        throw new IllegalArgumentException("lookup key cannot contain unit separator character U+001F; this character is reserved");
+      }
+    }
     final BytesRef utf8Key = new BytesRef(key);
     try {
 
@@ -720,7 +698,7 @@ public class AnalyzingSuggester extends 
 
       final List<LookupResult> results = new ArrayList<LookupResult>();
 
-      List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(lookupAutomaton, fst);
+      List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(convertAutomaton(lookupAutomaton), fst);
 
       if (exactFirst) {
 
@@ -864,6 +842,7 @@ public class AnalyzingSuggester extends 
     ts.close();
 
     replaceSep(automaton);
+    automaton = convertAutomaton(automaton);
 
     assert SpecialOperations.isFinite(automaton);
 

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Thu Jul 18 14:36:34 2013
@@ -15,16 +15,15 @@ package org.apache.lucene.search.suggest
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.FileOutputStream;
+
 import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadocs
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -33,6 +32,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 
@@ -54,6 +54,9 @@ import org.apache.lucene.util.fst.PairOu
  * #DEFAULT_NON_FUZZY_PREFIX} byte is not allowed to be
  * edited.  We allow up to 1 (@link
  * #DEFAULT_MAX_EDITS} edit.
+ * If {@link #unicodeAware} parameter in the constructor is set to true, maxEdits,
+ * minFuzzyLength, transpositions and nonFuzzyPrefix are measured in Unicode code 
+ * points (actual letters) instead of bytes. 
  *
  * <p>
  * NOTE: This suggester does not boost suggestions that
@@ -66,12 +69,20 @@ import org.apache.lucene.util.fst.PairOu
  * like synonyms to keep the complexity of the prefix intersection low for good
  * lookup performance. At index time, complex analyzers can safely be used.
  * </p>
+ *
+ * @lucene.experimental
  */
 public final class FuzzySuggester extends AnalyzingSuggester {
   private final int maxEdits;
   private final boolean transpositions;
   private final int nonFuzzyPrefix;
   private final int minFuzzyLength;
+  private final boolean unicodeAware;
+
+  /** Measure maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix 
+   *  parameters in Unicode code points (actual letters)
+   *  instead of bytes. */
+  public static final boolean DEFAULT_UNICODE_AWARE = false;
 
   /**
    * The default minimum length of the key passed to {@link
@@ -114,7 +125,7 @@ public final class FuzzySuggester extend
    */
   public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
     this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS,
-         DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH);
+         DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE);
   }
 
   /**
@@ -138,11 +149,12 @@ public final class FuzzySuggester extend
    *        Levenshtein algorithm.
    * @param nonFuzzyPrefix length of common (non-fuzzy) prefix (see default {@link #DEFAULT_NON_FUZZY_PREFIX}
    * @param minFuzzyLength minimum length of lookup key before any edits are allowed (see default {@link #DEFAULT_MIN_FUZZY_LENGTH})
+   * @param unicodeAware operate Unicode code points instead of bytes.
    */
   public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer,
                         int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
                         int maxEdits, boolean transpositions, int nonFuzzyPrefix,
-                        int minFuzzyLength) {
+                        int minFuzzyLength, boolean unicodeAware) {
     super(indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions);
     if (maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       throw new IllegalArgumentException("maxEdits must be between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE);
@@ -158,6 +170,7 @@ public final class FuzzySuggester extend
     this.transpositions = transpositions;
     this.nonFuzzyPrefix = nonFuzzyPrefix;
     this.minFuzzyLength = minFuzzyLength;
+    this.unicodeAware = unicodeAware;
   }
   
   @Override
@@ -176,7 +189,7 @@ public final class FuzzySuggester extend
     // "compete") ... in which case I think the wFST needs
     // to be log weights or something ...
 
-    Automaton levA = toLevenshteinAutomata(lookupAutomaton);
+    Automaton levA = convertAutomaton(toLevenshteinAutomata(lookupAutomaton));
     /*
       Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
       w.write(levA.toDot());
@@ -186,6 +199,24 @@ public final class FuzzySuggester extend
     return FSTUtil.intersectPrefixPaths(levA, fst);
   }
 
+  @Override
+  protected Automaton convertAutomaton(Automaton a) {
+    if (unicodeAware) {
+      Automaton utf8automaton = new UTF32ToUTF8().convert(a);
+      BasicOperations.determinize(utf8automaton);
+      return utf8automaton;
+    } else {
+      return a;
+    }
+  }
+
+  @Override
+  TokenStreamToAutomaton getTokenStreamToAutomaton() {
+    final TokenStreamToAutomaton tsta = super.getTokenStreamToAutomaton();
+    tsta.setUnicodeArcs(unicodeAware);
+    return tsta;
+  }
+
   Automaton toLevenshteinAutomata(Automaton automaton) {
     final Set<IntsRef> ref = SpecialOperations.getFiniteStrings(automaton, -1);
     Automaton subs[] = new Automaton[ref.size()];
@@ -203,7 +234,7 @@ public final class FuzzySuggester extend
         // to allow the trailing dedup bytes to be
         // edited... but then 0 byte is "in general" allowed
         // on input (but not in UTF8).
-        LevenshteinAutomata lev = new LevenshteinAutomata(ints, 255, transpositions);
+        LevenshteinAutomata lev = new LevenshteinAutomata(ints, unicodeAware ? Character.MAX_CODE_POINT : 255, transpositions);
         Automaton levAutomaton = lev.toAutomaton(maxEdits);
         Automaton combined = BasicOperations.concatenate(Arrays.asList(prefix, levAutomaton));
         combined.setDeterministic(true); // its like the special case in concatenate itself, except we cloneExpanded already

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java Thu Jul 18 14:36:34 2013
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Reader;
-import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -48,8 +47,6 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.TermFreq;
 import org.apache.lucene.search.suggest.TermFreqArrayIterator;
@@ -594,7 +591,7 @@ public class AnalyzingSuggesterTest exte
     }
   }
 
-  private static char SEP = '\uFFFF';
+  private static char SEP = '\u001F';
 
   public void testRandom() throws Exception {
 
@@ -822,70 +819,6 @@ public class AnalyzingSuggesterTest exte
     }
   }
 
-  public void testStolenBytes() throws Exception {
-
-    // First time w/ preserveSep, second time without:
-    for(int i=0;i<2;i++) {
-      
-      final Analyzer analyzer = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-            Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
-        
-            // TokenStream stream = new SynonymFilter(tokenizer, map, true);
-            // return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
-            return new TokenStreamComponents(tokenizer) {
-              int tokenStreamCounter = 0;
-              final TokenStream[] tokenStreams = new TokenStream[] {
-                new CannedBinaryTokenStream(new BinaryToken[] {
-                    token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
-                  }),
-                new CannedTokenStream(new Token[] {
-                    token("a",1,1),          
-                    token("a",1,1)
-                  }),
-                new CannedTokenStream(new Token[] {
-                    token("a",1,1),
-                    token("a",1,1)
-                  }),
-                new CannedBinaryTokenStream(new BinaryToken[] {
-                    token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
-                  })
-              };
-
-              @Override
-              public TokenStream getTokenStream() {
-                TokenStream result = tokenStreams[tokenStreamCounter];
-                tokenStreamCounter++;
-                return result;
-              }
-         
-              @Override
-              protected void setReader(final Reader reader) throws IOException {
-              }
-            };
-          }
-        };
-
-      TermFreq keys[] = new TermFreq[] {
-        new TermFreq("a a", 50),
-        new TermFreq("a b", 50),
-      };
-
-      AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer, analyzer, AnalyzingSuggester.EXACT_FIRST | (i==0 ? AnalyzingSuggester.PRESERVE_SEP : 0), 256, -1);
-      suggester.build(new TermFreqArrayIterator(keys));
-      List<LookupResult> results = suggester.lookup("a a", false, 5);
-      assertEquals(1, results.size());
-      assertEquals("a b", results.get(0).key);
-      assertEquals(50, results.get(0).value);
-
-      results = suggester.lookup("a a", false, 5);
-      assertEquals(1, results.size());
-      assertEquals("a a", results.get(0).key);
-      assertEquals(50, results.get(0).value);
-    }
-  }
-
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
     AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 2, -1);
@@ -1193,4 +1126,24 @@ public class AnalyzingSuggesterTest exte
     suggester.build(new TermFreqArrayIterator(new TermFreq[] {new TermFreq("a", 1)}));
     assertEquals("[a/1]", suggester.lookup("a", false, 1).toString());
   }
+  
+  public void testIllegalLookupArgument() throws Exception {
+    Analyzer a = new MockAnalyzer(random());
+    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1);
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+        new TermFreq("а где Люси?", 7),
+    }));
+    try {
+      suggester.lookup("а\u001E", false, 3);
+      fail("should throw IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+    try {
+      suggester.lookup("а\u001F", false, 3);
+      fail("should throw IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java Thu Jul 18 14:36:34 2013
@@ -60,7 +60,9 @@ public class FuzzySuggesterTest extends 
       keys.add(new TermFreq("boo" + _TestUtil.randomSimpleString(random()), 1 + random().nextInt(100)));
     }
     keys.add(new TermFreq("foo bar boo far", 12));
-    FuzzySuggester suggester = new FuzzySuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
+    MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
+    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+                                                  0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, FuzzySuggester.DEFAULT_UNICODE_AWARE);
     suggester.build(new TermFreqArrayIterator(keys));
     int numIters = atLeast(10);
     for (int i = 0; i < numIters; i++) {
@@ -72,6 +74,27 @@ public class FuzzySuggesterTest extends 
     }
   }
   
+  public void testNonLatinRandomEdits() throws IOException {
+    List<TermFreq> keys = new ArrayList<TermFreq>();
+    int numTerms = atLeast(100);
+    for (int i = 0; i < numTerms; i++) {
+      keys.add(new TermFreq("буу" + _TestUtil.randomSimpleString(random()), 1 + random().nextInt(100)));
+    }
+    keys.add(new TermFreq("фуу бар буу фар", 12));
+    MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
+    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+        0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, true);
+    suggester.build(new TermFreqArrayIterator(keys));
+    int numIters = atLeast(10);
+    for (int i = 0; i < numIters; i++) {
+      String addRandomEdit = addRandomEdit("фуу бар буу", 0);
+      List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence(addRandomEdit, random()), false, 2);
+      assertEquals(addRandomEdit, 1, results.size());
+      assertEquals("фуу бар буу фар", results.get(0).key.toString());
+      assertEquals(12, results.get(0).value, 0.01F);
+    }
+  }
+
   /** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
   public void testKeyword() throws Exception {
     TermFreq keys[] = new TermFreq[] {
@@ -185,7 +208,7 @@ public class FuzzySuggesterTest extends 
     int options = 0;
 
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, 1, true, 1, 3, false);
     suggester.build(new TermFreqArrayIterator(keys));
     // TODO: would be nice if "ab " would allow the test to
     // pass, and more generally if the analyzer can know
@@ -394,7 +417,7 @@ public class FuzzySuggesterTest extends 
   public void testExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3, false);
     suggester.build(new TermFreqArrayIterator(new TermFreq[] {
           new TermFreq("x y", 1),
           new TermFreq("x y z", 3),
@@ -433,7 +456,7 @@ public class FuzzySuggesterTest extends 
   public void testNonExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3, false);
 
     suggester.build(new TermFreqArrayIterator(new TermFreq[] {
           new TermFreq("x y", 1),
@@ -580,12 +603,13 @@ public class FuzzySuggesterTest extends 
     TermFreq[] keys = new TermFreq[numQueries];
 
     boolean preserveSep = random().nextBoolean();
+    boolean unicodeAware = random().nextBoolean();
 
     final int numStopChars = random().nextInt(10);
     final boolean preserveHoles = random().nextBoolean();
 
     if (VERBOSE) {
-      System.out.println("TEST: " + numQueries + " words; preserveSep=" + preserveSep + " numStopChars=" + numStopChars + " preserveHoles=" + preserveHoles);
+      System.out.println("TEST: " + numQueries + " words; preserveSep=" + preserveSep + " ; unicodeAware=" + unicodeAware + " numStopChars=" + numStopChars + " preserveHoles=" + preserveHoles);
     }
     
     for (int i = 0; i < numQueries; i++) {
@@ -606,7 +630,7 @@ public class FuzzySuggesterTest extends 
               if (token > 0) {
                 key += " ";
               }
-              if (preserveSep && analyzedKey.length() > 0 && analyzedKey.charAt(analyzedKey.length()-1) != ' ') {
+              if (preserveSep && analyzedKey.length() > 0 && (unicodeAware ? analyzedKey.codePointAt(analyzedKey.codePointCount(0, analyzedKey.length())-1) != ' ' : analyzedKey.charAt(analyzedKey.length()-1) != ' ')) {
                 analyzedKey += " ";
               }
               key += s;
@@ -659,7 +683,7 @@ public class FuzzySuggesterTest extends 
 
     Analyzer a = new MockTokenEatingAnalyzer(numStopChars, preserveHoles);
     FuzzySuggester suggester = new FuzzySuggester(a, a,
-                                                  preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, 1, false, 1, 3);
+                                                  preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, 1, false, 1, 3, unicodeAware);
     suggester.build(new TermFreqArrayIterator(keys));
 
     for (String prefix : allPrefixes) {
@@ -728,7 +752,7 @@ public class FuzzySuggesterTest extends 
       // us the "answer key" (ie maybe we have a bug in
       // suggester.toLevA ...) ... but testRandom2() fixes
       // this:
-      Automaton automaton = suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey));
+      Automaton automaton = suggester.convertAutomaton(suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey)));
       assertTrue(automaton.isDeterministic());
       // TODO: could be faster... but its slowCompletor for a reason
       BytesRef spare = new BytesRef();
@@ -799,7 +823,7 @@ public class FuzzySuggesterTest extends 
 
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, 1, true, 1, 3, false);
 
     List<TermFreq> keys = Arrays.asList(new TermFreq[] {
         new TermFreq("a", 40),
@@ -820,7 +844,7 @@ public class FuzzySuggesterTest extends 
 
   public void testEditSeps() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, 2, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, 2, true, 1, 3, false);
 
     List<TermFreq> keys = Arrays.asList(new TermFreq[] {
         new TermFreq("foo bar", 40),
@@ -878,7 +902,8 @@ public class FuzzySuggesterTest extends 
             // NOTE: can only use ascii here so that, in
             // UTF8 byte space it's still a single
             // insertion:
-            int x = random().nextInt(128);
+            // bytes 0x1e and 0x1f are reserved
+            int x = random().nextBoolean() ? random().nextInt(30) :  32 + random().nextInt(128 - 32);
             builder.append((char) x);
             for (int j = i; j < input.length; j++) {
               builder.append(input[j]);  
@@ -933,7 +958,7 @@ public class FuzzySuggesterTest extends 
     boolean transpositions = random().nextBoolean();
     // TODO: test graph analyzers
     // TODO: test exactFirst / preserveSep permutations
-    FuzzySuggester suggest = new FuzzySuggester(a, a, 0, 256, -1, maxEdits, transpositions, prefixLen, prefixLen);
+    FuzzySuggester suggest = new FuzzySuggester(a, a, 0, 256, -1, maxEdits, transpositions, prefixLen, prefixLen, false);
 
     if (VERBOSE) {
       System.out.println("TEST: maxEdits=" + maxEdits + " prefixLen=" + prefixLen + " transpositions=" + transpositions + " num=" + NUM);

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java?rev=1504490&r1=1504489&r2=1504490&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/FuzzyLookupFactory.java Thu Jul 18 14:36:34 2013
@@ -33,23 +33,33 @@ import org.apache.solr.spelling.suggest.
 public class FuzzyLookupFactory extends LookupFactory {
 
   /**
+   * If <code>true</code>, maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix 
+   * will be measured in Unicode code points (actual letters) instead of bytes.
+   */
+  public static final String UNICODE_AWARE = "unicodeAware";
+
+  /**
    * Maximum number of edits allowed, used by {@link LevenshteinAutomata#toAutomaton(int)}
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String MAX_EDITS = "maxEdits";
   
   /**
    * If transpositions are allowed, Fuzzy suggestions will be computed based on a primitive 
    * edit operation. If it is false, it will be based on the classic Levenshtein algorithm.
+   * Transpositions of bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String TRANSPOSITIONS = "transpositions";
   
   /**
    * Length of common (non-fuzzy) prefix for the suggestions
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String NON_FUZZY_PREFIX = "nonFuzzyPrefix";
   
   /**
    * Minimum length of lookup key before any edits are allowed for the suggestions
+   * in bytes or Unicode code points (if {@link #UNICODE_AWARE} option is set to true).
    */
   public static final String MIN_FUZZY_LENGTH = "minFuzzyLength";
   
@@ -113,9 +123,13 @@ public class FuzzyLookupFactory extends 
     ? Integer.parseInt(params.get(MIN_FUZZY_LENGTH).toString())
     :FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH;
     
+    boolean unicodeAware = (params.get(UNICODE_AWARE) != null)
+    ? Boolean.valueOf(params.get(UNICODE_AWARE).toString())
+    : FuzzySuggester.DEFAULT_UNICODE_AWARE;
+    
     return new FuzzySuggester(indexAnalyzer, queryAnalyzer, options, 
         maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, maxEdits, 
-        transpositions, nonFuzzyPrefix, minFuzzyLength);
+        transpositions, nonFuzzyPrefix, minFuzzyLength, unicodeAware);
   }
 
   @Override