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 2012/09/15 21:57:45 UTC

svn commit: r1385159 - in /lucene/dev/branches/lucene3842/lucene: core/src/java/org/apache/lucene/analysis/ core/src/java/org/apache/lucene/util/automaton/ core/src/java/org/apache/lucene/util/fst/ core/src/test/org/apache/lucene/analysis/ suggest/src/...

Author: mikemccand
Date: Sat Sep 15 19:57:44 2012
New Revision: 1385159

URL: http://svn.apache.org/viewvc?rev=1385159&view=rev
Log:
LUCENE-3842: add PRESERVE_SEP option

Modified:
    lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
    lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
    lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
    lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java
    lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java

Modified: lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java Sat Sep 15 19:57:44 2012
@@ -64,14 +64,10 @@ public class TokenStreamToAutomaton {
   }
 
   /** We create transition between two adjacent tokens. */
-  // nocommit should we ues 256?  ie, outside of the utf8
-  // byte range...
-  public static final int POS_SEP = 0;
+  public static final int POS_SEP = 256;
 
   /** We add this arc to represent a hole. */
-  // nocommit should we ues 257?  ie, outside of the utf8
-  // byte range...
-  public static final int HOLE = 1;
+  public static final int HOLE = 257;
 
   /** Pulls the graph (including {@link
    *  PositionLengthAttribute}) from the provided {@link

Modified: lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/automaton/State.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/automaton/State.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/automaton/State.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/automaton/State.java Sat Sep 15 19:57:44 2012
@@ -62,7 +62,7 @@ public class State implements Comparable
   /**
    * Resets transition set.
    */
-  final void resetTransitions() {
+  public final void resetTransitions() {
     transitionsArray = new Transition[0];
     numTransitions = 0;
   }
@@ -165,7 +165,11 @@ public class State implements Comparable
     }
   }
   
-  void addEpsilon(State to) {
+  /** Virtually adds an epsilon transition to the target
+   *  {@code to} state.  This is implemented by copying all
+   *  transitions from {@code to} to this state, and if {@code
+   *  to} is an accept state then set accept for this state. */
+  public void addEpsilon(State to) {
     if (to.accept) accept = true;
     for (Transition t : to.getTransitions())
       addTransition(t);

Modified: lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Sat Sep 15 19:57:44 2012
@@ -836,7 +836,9 @@ public final class Util {
   public static BytesRef toBytesRef(IntsRef input, BytesRef scratch) {
     scratch.grow(input.length);
     for(int i=0;i<input.length;i++) {
-      scratch.bytes[i] = (byte) input.ints[i+input.offset];
+      int value = input.ints[i+input.offset];
+      assert value >= Byte.MIN_VALUE && value <= Byte.MAX_VALUE: "value " + value + " doesn't fit into byte";
+      scratch.bytes[i] = (byte) value;
     }
     scratch.length = input.length;
     return scratch;

Modified: lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Sat Sep 15 19:57:44 2012
@@ -431,12 +431,26 @@ public class TestGraphTokenizers extends
     System.out.println("TEST: saved to /x/tmp3/out.dot");
   }
 
-  private static final Automaton SEP_A = BasicAutomata.makeCharRange(TokenStreamToAutomaton.POS_SEP,
-                                                                     TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton SEP_A = BasicAutomata.makeChar(TokenStreamToAutomaton.POS_SEP);
+  private static final Automaton HOLE_A = BasicAutomata.makeChar(TokenStreamToAutomaton.HOLE);
 
-  private static final String SEP = "" + (char) TokenStreamToAutomaton.POS_SEP;
+  private Automaton join(String ... strings) {
+    List<Automaton> as = new ArrayList<Automaton>();
+    for(String s : strings) {
+      as.add(BasicAutomata.makeString(s));
+      as.add(SEP_A);
+    }
+    as.remove(as.size()-1);
+    return BasicOperations.concatenate(as);
+  }
+
+  private Automaton join(Automaton ... as) {
+    return BasicOperations.concatenate(Arrays.asList(as));
+  }
 
-  private static final String HOLE = "" + (char) TokenStreamToAutomaton.HOLE;
+  private Automaton s2a(String s) {
+    return BasicAutomata.makeString(s);
+  }
 
   public void testTwoTokens() throws Exception {
 
@@ -446,7 +460,7 @@ public class TestGraphTokenizers extends
         token("def", 1, 1),
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
-    final Automaton expected =  BasicAutomata.makeString("abc" + SEP + "def");
+    final Automaton expected =  join("abc", "def");
 
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
@@ -461,7 +475,7 @@ public class TestGraphTokenizers extends
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
 
-    final Automaton expected = BasicAutomata.makeString("abc" + SEP + HOLE + SEP + "def");
+    final Automaton expected = join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"));
 
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
@@ -492,7 +506,7 @@ public class TestGraphTokenizers extends
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
     final Automaton a1 = BasicAutomata.makeString("xyz");
-    final Automaton a2 = BasicAutomata.makeString("abc" + SEP + "def");
+    final Automaton a2 = join("abc", "def");
                                                                    
     final Automaton expected = BasicOperations.union(a1, a2);
     //toDot(actual);
@@ -509,10 +523,10 @@ public class TestGraphTokenizers extends
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
     final Automaton a1 = BasicOperations.union(
-                                               BasicAutomata.makeString("a" + SEP + HOLE),
+                                               join(s2a("a"), SEP_A, HOLE_A),
                                                BasicAutomata.makeString("X"));
     final Automaton expected = BasicOperations.concatenate(a1,
-                                                           BasicAutomata.makeString(SEP + "b"));
+                                                           join(SEP_A, s2a("b")));
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
   }
@@ -527,7 +541,7 @@ public class TestGraphTokenizers extends
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
     final Automaton expected = BasicOperations.union(
-                                                     BasicAutomata.makeString("xyz" + SEP + HOLE + SEP + "def"),
+                                                     join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")),
                                                      BasicAutomata.makeString("abc"));
     assertTrue(BasicOperations.sameLanguage(expected, actual));
   }
@@ -543,7 +557,7 @@ public class TestGraphTokenizers extends
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
     final Automaton a1 = BasicAutomata.makeString("xyz");
-    final Automaton a2 = BasicAutomata.makeString("abc" + SEP + "def" + SEP + "ghi");
+    final Automaton a2 = join("abc", "def", "ghi");
     final Automaton expected = BasicOperations.union(a1, a2);
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
@@ -562,7 +576,7 @@ public class TestGraphTokenizers extends
         token("abc", 2, 1),
       });
     final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
-    final Automaton expected = BasicAutomata.makeString(HOLE + SEP + "abc");
+    final Automaton expected = join(HOLE_A, SEP_A, s2a("abc"));
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
   }

Modified: lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java (original)
+++ lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java Sat Sep 15 19:57:44 2012
@@ -48,16 +48,18 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.State;
+import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
-import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.FST.Arc;
 import org.apache.lucene.util.fst.FST.BytesReader;
-import org.apache.lucene.util.fst.PairOutputs;
+import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
+import org.apache.lucene.util.fst.PairOutputs;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
-import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.fst.Util.MinResult;
+import org.apache.lucene.util.fst.Util;
 
 /**
  * Suggester that first analyzes the surface form, adds the
@@ -105,25 +107,82 @@ public class AnalyzingCompletionLookup e
    */
   private final boolean exactFirst;
   
+  /** 
+   * True if separator between tokens should be preservered.
+   */
+  private final boolean preserveSep;
+
   /**
-   * Calls {@link #AnalyzingCompletionLookup(Analyzer,boolean) AnalyzingCompletionLookup(analyzer, true)}
+   * Calls {@link #AnalyzingCompletionLookup(Analyzer,int) AnalyzingCompletionLookup(analyzer, EXACT_FIRST | PRESERVE_SEP)}
    */
   public AnalyzingCompletionLookup(Analyzer analyzer) {
-    this(analyzer, true);
+    this(analyzer, EXACT_FIRST | PRESERVE_SEP);
   }
-  
+
+  /** Include this flag in the options parameter {@link
+   * #AnalyzingCompletionLookup(Analyzer,int)} to always
+   * return exact matches first regardless of score.  This
+   * has no performance impact but could result in
+   * low-quality suggestions. */
+  public static final int EXACT_FIRST = 1;
+
+  /** Include this flag in the options parameter {@link
+   * #AnalyzingCompletionLookup(Analyzer,int)} to preserve
+   * token separators when matching. */
+  public static final int PRESERVE_SEP = 2;
+
+  /** Represents the separation between tokens, if
+   *  PRESERVE_SEP was specified */
+  private static final byte SEP_BYTE = 0;
+
   /**
    * Creates a new suggester.
    * 
    * @param analyzer Analyzer that will be used for analyzing suggestions.
-   * @param exactFirst <code>true</code> if suggestions that match the 
-   *        prefix exactly should always be returned first, regardless
-   *        of score. This has no performance impact, but could result
-   *        in low-quality suggestions.
+   * @param options see {@link #EXACT_FIRST}, {@link #PRESERVE_SEP}
    */
-  public AnalyzingCompletionLookup(Analyzer analyzer, boolean exactFirst) {
+  public AnalyzingCompletionLookup(Analyzer analyzer, int options) {
     this.analyzer = analyzer;
-    this.exactFirst = exactFirst;
+    if ((options & ~(EXACT_FIRST | PRESERVE_SEP)) != 0) {
+      throw new IllegalArgumentException("options should only contain EXACT_FIRST and PRESERVE_SEP; got " + options);
+    }
+    this.exactFirst = (options & EXACT_FIRST) != 0;
+    this.preserveSep = (options & PRESERVE_SEP) != 0;
+  }
+
+  // Replaces SEP with epsilon or remaps them if
+  // we were asked to preserve them:
+  private void replaceSep(Automaton a) {
+
+    State[] states = a.getNumberedStates();
+
+    // Go in reverse topo sort so we know we only have to
+    // make one pass:
+    for(int stateNumber=states.length-1;stateNumber >=0;stateNumber--) {
+      final State state = states[stateNumber];
+      List<Transition> newTransitions = new ArrayList<Transition>();
+      for(Transition t : state.getTransitions()) {
+        assert t.getMin() == t.getMax();
+        if (t.getMin() == TokenStreamToAutomaton.POS_SEP) {
+          if (preserveSep) {
+            // Remap to SEP_BYTE:
+            t = new Transition(SEP_BYTE, t.getDest());
+          } else {
+            // NOTE: sort of weird because this will grow
+            // the transition array we are iterating over,
+            // but because we are going in reverse topo sort
+            // it will not add any SEP/HOLE transitions:
+            state.addEpsilon(t.getDest());
+            t = null;
+          }
+        }
+        if (t != null) {
+          newTransitions.add(t);
+        }
+      }
+      state.resetTransitions();
+      state.setTransitions(newTransitions.toArray(new Transition[newTransitions.size()]));
+    }
   }
   
   @Override
@@ -137,8 +196,6 @@ public class AnalyzingCompletionLookup e
     Sort.ByteSequencesReader reader = null;
     BytesRef scratch = new BytesRef();
 
-    assert TokenStreamToAutomaton.POS_SEP < Byte.MAX_VALUE;
-
     BytesRef separator = new BytesRef(new byte[] { (byte)TokenStreamToAutomaton.POS_SEP });
     
     // analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short) 
@@ -158,6 +215,9 @@ public class AnalyzingCompletionLookup e
         Automaton automaton = TokenStreamToAutomaton.toAutomaton(ts);
         ts.end();
         ts.close();
+
+        replaceSep(automaton);
+
         assert SpecialOperations.isFinite(automaton);
 
         // Get all paths from the automaton (there can be
@@ -166,7 +226,7 @@ public class AnalyzingCompletionLookup e
 
         // nocommit: we should probably not wire this param to -1 but have a reasonable limit?!
         Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, -1);
-        for (IntsRef path : paths) {        
+        for (IntsRef path : paths) {
 
           Util.toBytesRef(path, scratch);
           
@@ -203,6 +263,7 @@ public class AnalyzingCompletionLookup e
       BytesRef surface = new BytesRef();
       IntsRef scratchInts = new IntsRef();
       ByteArrayDataInput input = new ByteArrayDataInput();
+      int dedup = 0;
       while (reader.read(scratch)) {
         input.reset(scratch.bytes, scratch.offset, scratch.length);
         input.setPosition(input.length()-2);
@@ -226,8 +287,21 @@ public class AnalyzingCompletionLookup e
           // increasing bytes (it wont matter) ... or we
           // could use multiple outputs for a single input?
           // this would be more efficient?
-          continue;
+          if (dedup < 256) {
+            analyzed.grow(analyzed.length+1);
+            analyzed.bytes[analyzed.length] = (byte) dedup;
+            dedup++;
+            analyzed.length++;
+          } else {
+            // nocommit add param to limit "dups"???
+
+            // More than 256 dups: skip the rest:
+            continue;
+          }
+        } else {
+          dedup = 0;
         }
+
         Util.toIntsRef(analyzed, scratchInts);
         // nocommit (why?)
         builder.add(scratchInts, outputs.newPair(cost, BytesRef.deepCopyOf(surface)));
@@ -289,6 +363,8 @@ public class AnalyzingCompletionLookup e
       throw new RuntimeException(bogus);
     }
 
+    replaceSep(automaton);
+
     // TODO: we can optimize this somewhat by determinizing
     // while we convert
     automaton = Automaton.minimize(automaton);
@@ -307,7 +383,11 @@ public class AnalyzingCompletionLookup e
       throw new RuntimeException(bogus);
     }
 
-    // nocommit maybe nuke exactFirst...? but... it's useful?
+    // nocommit maybe nuke exactFirst...? but... it's
+    // useful?
+    // nocommit: exactFirst is not well defined here ... the
+    // "exactness" refers to the analyzed form not the
+    // surface form
     if (exactFirst) {
       for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
         if (path.fstNode.isFinal()) {

Modified: lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java?rev=1385159&r1=1385158&r2=1385159&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java (original)
+++ lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java Sat Sep 15 19:57:44 2012
@@ -120,7 +120,31 @@ public class AnalyzingCompletionTest ext
     assertEquals("the ghost of christmas past", results.get(0).key.toString());
     assertEquals(50, results.get(0).value, 0.01F);
   }
-  
+
+  public void testNoSeps() throws Exception {
+    TermFreq[] keys = new TermFreq[] {
+      new TermFreq("ab cd", 0),
+      new TermFreq("abcd", 1),
+    };
+
+    int options = 0;
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random()), options);
+    suggester.build(new TermFreqArrayIterator(keys));
+    // nocommit if i change this to "ab " ... the test fails
+    // now but really it should pass???  problem is
+    // analyzers typically strip trailing space?  really we
+    // need a SEP token appear instead...?  hmm actually i
+    // think we need to look @ posIncAtt after .end()?
+    List<LookupResult> r = suggester.lookup(_TestUtil.stringToCharSequence("ab c", random()), false, 2);
+    assertEquals(2, r.size());
+
+    // With no PRESERVE_SEPS specified, "ab c" should also
+    // complete to "abcd", which has higher weight so should
+    // appear first:
+    assertEquals("abcd", r.get(0).key.toString());
+  }
+
   public void testInputPathRequired() throws Exception {
     TermFreq keys[] = new TermFreq[] {
         new TermFreq("ab xc", 50),
@@ -238,7 +262,10 @@ public class AnalyzingCompletionTest ext
       keys[i] = new TermFreq(s, weight);
     }
 
-    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false), false);
+    // nocommit also test NOT preserving seps/holes
+    // nocommit why no failure if we DON'T preserve seps/holes...?
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false),
+                                                                        AnalyzingCompletionLookup.PRESERVE_SEP);
     suggester.build(new TermFreqArrayIterator(keys));
     
     for (String prefix : allPrefixes) {